diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java index 72739ef265..e5b7eeff73 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java @@ -227,21 +227,32 @@ void testComplexTypeFetch() throws Exception { new Object[] { 1, new String[] {"a", "b"}, - new Object[] {new int[] {1, 2}, new int[] {3, 4}} + new Object[] {new int[] {1, 2}, new int[] {3, 4}}, + new Object[] {10, new Object[] {20, "nested"}, "row1"} }, new Object[] { - 2, new String[] {"c", null}, new Object[] {null, new int[] {3, 4}} + 2, + new String[] {"c", null}, + new Object[] {null, new int[] {3, 4}}, + new Object[] {30, new Object[] {40, "test"}, "row2"} }, new Object[] { 3, new String[] {"e", "f"}, - new Object[] {new int[] {5, 6, 7}, new int[] {8}} + new Object[] {new int[] {5, 6, 7}, new int[] {8}}, + new Object[] {50, new Object[] {60, "value"}, "row3"} }); Schema schema = Schema.newBuilder() .column("a", DataTypes.INT()) .column("b", DataTypes.ARRAY(DataTypes.STRING())) .column("c", DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT()))) + .column( + "d", + DataTypes.ROW( + DataTypes.INT(), + DataTypes.ROW(DataTypes.INT(), DataTypes.STRING()), + DataTypes.STRING())) .build(); TableInfo tableInfo = TableInfo.of( @@ -299,6 +310,17 @@ void testComplexTypeFetch() throws Exception { .isEqualTo(Arrays.deepToString((Object[]) complexData.get(i)[1])); assertThat(row.getArray(2).toString()) .isEqualTo(Arrays.deepToString((Object[]) complexData.get(i)[2])); + InternalRow nestedRow = row.getRow(3, 3); + assertThat(nestedRow).isNotNull(); + assertThat(nestedRow.getInt(0)).isEqualTo(((Object[]) complexData.get(i)[3])[0]); + InternalRow deeplyNestedRow = nestedRow.getRow(1, 2); + assertThat(deeplyNestedRow).isNotNull(); + assertThat(deeplyNestedRow.getInt(0)) + .isEqualTo(((Object[]) ((Object[]) complexData.get(i)[3])[1])[0]); + assertThat(deeplyNestedRow.getString(1).toString()) + .isEqualTo(((Object[]) ((Object[]) complexData.get(i)[3])[1])[1]); + assertThat(nestedRow.getString(2).toString()) + .isEqualTo(((Object[]) complexData.get(i)[3])[2]); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/AbstractBinaryWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/AbstractBinaryWriter.java index 7258b048ef..c5a83da444 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/AbstractBinaryWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/AbstractBinaryWriter.java @@ -19,6 +19,7 @@ import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.row.serializer.ArraySerializer; +import org.apache.fluss.row.serializer.RowSerializer; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -88,20 +89,20 @@ public void writeArray(int pos, InternalArray input, ArraySerializer serializer) pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); } - // TODO: Map and Row write methods will be added in Issue #1973 and #1974 + // TODO: Map and Row write methods will be added in Issue #1973 // @Override // public void writeMap(int pos, InternalMap input, InternalMapSerializer serializer) { // BinaryMap binary = serializer.toBinaryMap(input); // writeSegmentsToVarLenPart( // pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); // } - // - // @Override - // public void writeRow(int pos, InternalRow value, InternalRowSerializer serializer) { - // BinaryRow binary = serializer.toBinaryRow(value); - // writeSegmentsToVarLenPart( - // pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); - // } + + @Override + public void writeRow(int pos, InternalRow value, RowSerializer serializer) { + BinaryRow binary = serializer.toBinaryRow(value); + writeSegmentsToVarLenPart( + pos, binary.getSegments(), binary.getOffset(), binary.getSizeInBytes()); + } @Override public void writeChar(int pos, BinaryString value, int length) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java b/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java index 881b3c96f6..0aef1b0b6c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/BinaryArray.java @@ -19,6 +19,7 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.types.DataType; import java.lang.reflect.Array; @@ -48,7 +49,7 @@ * @since 0.9 */ @PublicEvolving -public final class BinaryArray extends BinarySection +public abstract class BinaryArray extends BinarySection implements InternalArray, MemoryAwareGetters, DataSetters { private static final long serialVersionUID = 1L; @@ -111,9 +112,7 @@ public static int calculateFixLengthPartSize(DataType type) { /** The position to start storing array elements. */ private transient int elementOffset; - public BinaryArray() {} - - private void assertIndexIsValid(int ordinal) { + protected void assertIndexIsValid(int ordinal) { assert ordinal >= 0 : "ordinal (" + ordinal + ") should >= 0"; assert ordinal < size : "ordinal (" + ordinal + ") should < " + size; } @@ -265,11 +264,14 @@ public TimestampLtz getTimestampLtz(int pos, int precision) { @Override public InternalArray getArray(int pos) { assertIndexIsValid(pos); - return BinarySegmentUtils.readBinaryArray(segments, offset, getLong(pos)); + return BinarySegmentUtils.readBinaryArray( + segments, offset, getLong(pos), createNestedArrayInstance()); } + /** Creates a nested {@link BinaryArray} with the nested data type information. */ + protected abstract BinaryArray createNestedArrayInstance(); + // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 @Override public boolean getBoolean(int pos) { @@ -545,21 +547,26 @@ public T[] toObjectArray(DataType elementType) { return values; } - public BinaryArray copy() { - return copy(new BinaryArray()); - } - - public BinaryArray copy(BinaryArray reuse) { - byte[] bytes = BinarySegmentUtils.copyToBytes(segments, offset, sizeInBytes); - reuse.pointTo(MemorySegment.wrap(bytes), 0, sizeInBytes); - return reuse; - } - @Override public int hashCode() { return BinarySegmentUtils.hash(segments, offset, sizeInBytes); } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + // override equals and only checks the other object is instance of BinaryArray + if (!(o instanceof BinaryArray)) { + return false; + } + final BinarySection that = (BinarySection) o; + return sizeInBytes == that.sizeInBytes + && BinarySegmentUtils.equals( + segments, offset, that.segments, that.offset, sizeInBytes); + } + // ------------------------------------------------------------------------------------------ // Construction Utilities // ------------------------------------------------------------------------------------------ @@ -611,13 +618,13 @@ private static BinaryArray fromPrimitiveArray( UNSAFE.copyMemory( arr, offset, data, BYTE_ARRAY_BASE_OFFSET + headerInBytes, valueRegionInBytes); - BinaryArray result = new BinaryArray(); + BinaryArray result = new PrimitiveBinaryArray(); result.pointTo(MemorySegment.wrap(data), 0, (int) totalSize); return result; } public static BinaryArray fromLongArray(Long[] arr) { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, arr.length, 8); for (int i = 0; i < arr.length; i++) { Long v = arr[i]; @@ -636,7 +643,7 @@ public static BinaryArray fromLongArray(InternalArray arr) { return (BinaryArray) arr; } - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, arr.size(), 8); for (int i = 0; i < arr.size(); i++) { if (arr.isNullAt(i)) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/BinaryRow.java b/fluss-common/src/main/java/org/apache/fluss/row/BinaryRow.java index a7afaa8a36..2235a34a7c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/BinaryRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/BinaryRow.java @@ -18,6 +18,9 @@ package org.apache.fluss.row; import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.row.aligned.AlignedRow; +import org.apache.fluss.row.compacted.CompactedRow; +import org.apache.fluss.row.indexed.IndexedRow; /** * A binary format {@link InternalRow} that is backed on {@link MemorySegment} and supports all @@ -57,4 +60,20 @@ public interface BinaryRow extends InternalRow, MemoryAwareGetters { * @param sizeInBytes The size of the row. */ void pointTo(MemorySegment[] segments, int offset, int sizeInBytes); + + /** + * The binary row format types, it indicates the generated {@link BinaryRow} type by the {@link + * BinaryWriter}. + */ + enum BinaryRowFormat { + + /** Compacted binary row format, see {@link CompactedRow}. */ + COMPACTED, + + /** Aligned binary row format, see {@link AlignedRow}. */ + ALIGNED, + + /** Indexed binary row format, see {@link IndexedRow}. */ + INDEXED + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/BinarySegmentUtils.java b/fluss-common/src/main/java/org/apache/fluss/row/BinarySegmentUtils.java index 463d4781df..bd532634a2 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/BinarySegmentUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/BinarySegmentUtils.java @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.memory.OutputView; +import org.apache.fluss.row.aligned.AlignedRow; import org.apache.fluss.row.compacted.CompactedRow; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.types.DataType; @@ -989,14 +990,16 @@ public static Decimal readDecimal( return Decimal.fromUnscaledBytes(bytes, precision, scale); } - /** Gets an instance of {@link InternalArray} from underlying {@link MemorySegment}. */ + /** + * Read the array data into the reused {@link BinaryArray} instance from underlying {@link + * MemorySegment}. + */ public static BinaryArray readBinaryArray( - MemorySegment[] segments, int baseOffset, long offsetAndSize) { + MemorySegment[] segments, int baseOffset, long offsetAndSize, BinaryArray reusedArray) { final int size = ((int) offsetAndSize); int offset = (int) (offsetAndSize >> 32); - BinaryArray array = new BinaryArray(); - array.pointTo(segments, offset + baseOffset, size); - return array; + reusedArray.pointTo(segments, offset + baseOffset, size); + return reusedArray; } /** Read map data from segments. */ @@ -1013,27 +1016,33 @@ public static InternalMap readMap(MemorySegment[] segments, int offset, long num "Map type is not supported yet. Will be added in Issue #1973."); } - /** Read indexed row data from segments. */ + /** Read aligned row from segments. */ + public static InternalRow readAlignedRow( + MemorySegment[] segments, int baseOffset, long offsetAndSize, int numFields) { + final int size = ((int) offsetAndSize); + int offset = (int) (offsetAndSize >> 32); + AlignedRow row = new AlignedRow(numFields); + row.pointTo(segments, baseOffset + offset, size); + return row; + } + + /** Read indexed row from segments. */ public static InternalRow readIndexedRow( - MemorySegment[] segments, - int offset, - int numBytes, - int numFields, - DataType[] fieldTypes) { + MemorySegment[] segments, int baseOffset, long offsetAndSize, DataType[] fieldTypes) { + final int size = ((int) offsetAndSize); + int offset = (int) (offsetAndSize >> 32); IndexedRow row = new IndexedRow(fieldTypes); - row.pointTo(segments, offset, numBytes); + row.pointTo(segments, baseOffset + offset, size); return row; } - /** Read compacted row data from segments. */ + /** Read compacted row from segments. */ public static InternalRow readCompactedRow( - MemorySegment[] segments, - int offset, - int numBytes, - int numFields, - DataType[] fieldTypes) { + MemorySegment[] segments, int baseOffset, long offsetAndSize, DataType[] fieldTypes) { + final int size = ((int) offsetAndSize); + int offset = (int) (offsetAndSize >> 32); CompactedRow row = new CompactedRow(fieldTypes); - row.pointTo(segments, offset, numBytes); + row.pointTo(segments, baseOffset + offset, size); return row; } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java index 63c2290a12..9f1a110dc1 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/BinaryWriter.java @@ -18,9 +18,14 @@ package org.apache.fluss.row; import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.row.BinaryRow.BinaryRowFormat; import org.apache.fluss.row.serializer.ArraySerializer; +import org.apache.fluss.row.serializer.RowSerializer; import org.apache.fluss.types.ArrayType; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; + +import javax.annotation.Nullable; import java.io.Serializable; @@ -73,9 +78,7 @@ public interface BinaryWriter { void writeArray(int pos, InternalArray value, ArraySerializer serializer); - // TODO: Map and Row write methods will be added in Issue #1973 and #1974 - // void writeMap(int pos, InternalMap value, InternalMapSerializer serializer); - // void writeRow(int pos, InternalRow value, InternalRowSerializer serializer); + void writeRow(int pos, InternalRow value, RowSerializer serializer); /** Finally, complete write to set real size to binary. */ void complete(); @@ -87,9 +90,12 @@ public interface BinaryWriter { * Creates an accessor for setting the elements of a binary writer during runtime. * * @param elementType the element type + * @param rowFormat the binary row format, it is required when the element type has nested row + * type, otherwise, {@link IllegalArgumentException} will be thrown. */ - static BinaryWriter.ValueWriter createValueWriter(DataType elementType) { - BinaryWriter.ValueWriter valueWriter = createNotNullValueWriter(elementType); + static BinaryWriter.ValueWriter createValueWriter( + DataType elementType, BinaryRowFormat rowFormat) { + BinaryWriter.ValueWriter valueWriter = createNotNullValueWriter(elementType, rowFormat); if (!elementType.isNullable()) { return valueWriter; } @@ -108,7 +114,8 @@ static BinaryWriter.ValueWriter createValueWriter(DataType elementType) { * * @param elementType the element type */ - static BinaryWriter.ValueWriter createNotNullValueWriter(DataType elementType) { + static BinaryWriter.ValueWriter createNotNullValueWriter( + DataType elementType, @Nullable BinaryRowFormat rowFormat) { switch (elementType.getTypeRoot()) { case CHAR: int charLength = getLength(elementType); @@ -152,7 +159,7 @@ static BinaryWriter.ValueWriter createNotNullValueWriter(DataType elementType) { writer.writeTimestampLtz(pos, (TimestampLtz) value, timestampLtzPrecision); case ARRAY: final ArraySerializer arraySerializer = - new ArraySerializer(((ArrayType) elementType).getElementType()); + new ArraySerializer(((ArrayType) elementType).getElementType(), rowFormat); return (writer, pos, value) -> writer.writeArray(pos, (InternalArray) value, arraySerializer); @@ -161,9 +168,16 @@ static BinaryWriter.ValueWriter createNotNullValueWriter(DataType elementType) { throw new UnsupportedOperationException( "Map type is not supported yet. Will be added in Issue #1973."); case ROW: - // TODO: Row type support will be added in Issue #1974 - throw new UnsupportedOperationException( - "Row type is not supported yet. Will be added in Issue #1974."); + if (rowFormat == null) { + throw new IllegalArgumentException( + "Binary row format is required to write row."); + } + final RowType rowType = (RowType) elementType; + final RowSerializer rowSerializer = + new RowSerializer( + rowType.getFieldTypes().toArray(new DataType[0]), rowFormat); + return (writer, pos, value) -> + writer.writeRow(pos, (InternalRow) value, rowSerializer); default: String msg = String.format( diff --git a/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java b/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java index d3913e7ff1..2106e9ff66 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/DataGetters.java @@ -94,4 +94,7 @@ public interface DataGetters { /** Returns the array value at the given position. */ InternalArray getArray(int pos); + + /** Returns the row value at the given position. */ + InternalRow getRow(int pos, int numFields); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java b/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java index d97accd621..3e8a74a87c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/GenericArray.java @@ -219,7 +219,11 @@ public InternalArray getArray(int pos) { } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + return (InternalRow) getObject(pos); + } private Object getObject(int pos) { return ((Object[]) array)[pos]; diff --git a/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java b/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java index 5fc2b4ba86..eb53e87f10 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/GenericRow.java @@ -172,7 +172,11 @@ public InternalArray getArray(int pos) { } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + return (InternalRow) this.fields[pos]; + } @Override public boolean equals(Object o) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java b/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java index 1d16b738eb..0209b3cd51 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/InternalArray.java @@ -23,6 +23,7 @@ import org.apache.fluss.row.columnar.VectorizedColumnBatch; import org.apache.fluss.types.ArrayType; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; import javax.annotation.Nullable; @@ -132,10 +133,12 @@ static ElementGetter createElementGetter(DataType fieldType) { case ARRAY: elementGetter = InternalArray::getArray; break; + case ROW: + final int rowFieldCount = ((RowType) fieldType).getFieldCount(); + elementGetter = (array, pos) -> array.getRow(pos, rowFieldCount); + break; // TODO: MAP support will be added in Issue #1973 - // TODO: ROW support will be added in Issue #1974 case MAP: - case ROW: default: String msg = String.format( @@ -181,8 +184,24 @@ static ElementGetter createDeepElementGetter(DataType fieldType) { return new GenericArray(objs); }; break; - case MAP: case ROW: + RowType rowType = (RowType) fieldType; + int numFields = rowType.getFieldCount(); + InternalRow.FieldGetter[] fieldGetters = new InternalRow.FieldGetter[numFields]; + for (int i = 0; i < numFields; i++) { + fieldGetters[i] = InternalRow.createDeepFieldGetter(rowType.getTypeAt(i), i); + } + elementGetter = + (array, pos) -> { + InternalRow row = array.getRow(pos, numFields); + GenericRow genericRow = new GenericRow(numFields); + for (int i = 0; i < numFields; i++) { + genericRow.setField(i, fieldGetters[i].getFieldOrNull(row)); + } + return genericRow; + }; + break; + case MAP: String msg = String.format( "type %s not support in %s", diff --git a/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java b/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java index 16dfcb01f2..6efa54e009 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/InternalRow.java @@ -215,9 +215,10 @@ static FieldGetter createFieldGetter(DataType fieldType, int fieldPos) { fieldGetter = row -> row.getArray(fieldPos); break; // TODO: MAP support will be added in Issue #1973 - // TODO: ROW support will be added in Issue #1974 - case MAP: case ROW: + final int numFields = ((RowType) fieldType).getFieldCount(); + fieldGetter = row -> row.getRow(fieldPos, numFields); + break; default: throw new IllegalArgumentException("Illegal type: " + fieldType); } @@ -259,8 +260,25 @@ static FieldGetter createDeepFieldGetter(DataType fieldType, int fieldPos) { return new GenericArray(objs); }; break; - case MAP: case ROW: + RowType rowType = (RowType) fieldType; + int numFields = rowType.getFieldCount(); + FieldGetter[] nestedFieldGetters = new FieldGetter[numFields]; + for (int i = 0; i < numFields; i++) { + nestedFieldGetters[i] = createDeepFieldGetter(rowType.getTypeAt(i), i); + } + fieldGetter = + row -> { + InternalRow nestedRow = row.getRow(fieldPos, numFields); + GenericRow genericRow = new GenericRow(numFields); + for (int i = 0; i < numFields; i++) { + genericRow.setField( + i, nestedFieldGetters[i].getFieldOrNull(nestedRow)); + } + return genericRow; + }; + break; + case MAP: String msg = String.format( "type %s not support in %s", diff --git a/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java b/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java index db431f2acc..66b9b295f6 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/PaddingRow.java @@ -130,4 +130,9 @@ public byte[] getBytes(int pos) { public InternalArray getArray(int pos) { return row.getArray(pos); } + + @Override + public InternalRow getRow(int pos, int numFields) { + return row.getRow(pos, numFields); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java index d793f342ab..00e34615b8 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/ProjectedRow.java @@ -146,7 +146,11 @@ public InternalArray getArray(int pos) { } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + return row.getRow(indexMapping[pos], numFields); + } @Override public boolean equals(Object o) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/SequentialBinaryWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/SequentialBinaryWriter.java index 21fb9a1a67..485118688f 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/SequentialBinaryWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/SequentialBinaryWriter.java @@ -21,6 +21,7 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.memory.MemorySegment; import org.apache.fluss.row.serializer.ArraySerializer; +import org.apache.fluss.row.serializer.RowSerializer; /** * Writer for binary format in a sequential way. The write column position must be advanced in order @@ -69,7 +70,8 @@ public interface SequentialBinaryWriter extends BinaryWriter { // TODO: Map and Row write methods will be added in Issue #1973 and #1974 // void writeMap(InternalMap value, InternalMapSerializer serializer); - // void writeRow(InternalRow value, InternalRowSerializer serializer); + + void writeRow(InternalRow value, RowSerializer serializer); /** Finally, complete write to set real size to binary. */ void complete(); @@ -156,4 +158,9 @@ default void writeTimestampLtz(int pos, TimestampLtz value, int precision) { default void writeArray(int pos, InternalArray value, ArraySerializer serializer) { writeArray(value, serializer); } + + @Override + default void writeRow(int pos, InternalRow value, RowSerializer serializer) { + writeRow(value, serializer); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java index e185ed36bc..dad2b5822f 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/aligned/AlignedRow.java @@ -30,6 +30,7 @@ import org.apache.fluss.row.NullAwareGetters; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.array.AlignedArray; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DecimalType; import org.apache.fluss.types.LocalZonedTimestampType; @@ -390,11 +391,19 @@ public InternalArray getArray(int pos) { assertIndexIsValid(pos); int fieldOffset = getFieldOffset(pos); final long offsetAndSize = segments[0].getLong(fieldOffset); - return BinarySegmentUtils.readBinaryArray(segments, offset, offsetAndSize); + return BinarySegmentUtils.readBinaryArray( + segments, offset, offsetAndSize, new AlignedArray()); } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + assertIndexIsValid(pos); + int fieldOffset = getFieldOffset(pos); + final long offsetAndSize = segments[0].getLong(fieldOffset); + return BinarySegmentUtils.readAlignedRow(segments, offset, offsetAndSize, numFields); + } /** The bit is 1 when the field is null. Default is 0. */ public boolean anyNull() { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/array/AlignedArray.java b/fluss-common/src/main/java/org/apache/fluss/row/array/AlignedArray.java new file mode 100644 index 0000000000..9919f46b34 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/array/AlignedArray.java @@ -0,0 +1,42 @@ +/* + * 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.fluss.row.array; + +import org.apache.fluss.row.BinaryArray; +import org.apache.fluss.row.BinarySegmentUtils; +import org.apache.fluss.row.InternalRow; + +/** + * A {@link BinaryArray} that uses {@link org.apache.fluss.row.aligned.AlignedRow} as the binary + * format for arrays of nested row type. + */ +public class AlignedArray extends BinaryArray { + private static final long serialVersionUID = 1L; + + @Override + public InternalRow getRow(int pos, int numFields) { + return BinarySegmentUtils.readAlignedRow(segments, offset, getLong(pos), numFields); + } + + @Override + protected BinaryArray createNestedArrayInstance() { + return new AlignedArray(); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/array/CompactedArray.java b/fluss-common/src/main/java/org/apache/fluss/row/array/CompactedArray.java new file mode 100644 index 0000000000..566d74cc03 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/array/CompactedArray.java @@ -0,0 +1,70 @@ +/* + * 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.fluss.row.array; + +import org.apache.fluss.row.BinaryArray; +import org.apache.fluss.row.BinarySegmentUtils; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.ArrayType; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; + +/** + * A {@link BinaryArray} that uses {@link org.apache.fluss.row.compacted.CompactedRow} as the binary + * format for arrays of nested row type. + */ +public class CompactedArray extends BinaryArray { + + private final DataType elementType; + + private transient DataType[] nestedFields; + + public CompactedArray(DataType elementType) { + this.elementType = elementType; + } + + @Override + public InternalRow getRow(int pos, int numFields) { + assertIndexIsValid(pos); + if (elementType instanceof RowType) { + if (nestedFields == null) { + nestedFields = ((RowType) elementType).getFieldTypes().toArray(new DataType[0]); + } + if (nestedFields.length != numFields) { + throw new IllegalArgumentException( + "Unexpected number of fields " + numFields + " for " + elementType); + } + return BinarySegmentUtils.readCompactedRow( + segments, offset, getLong(pos), nestedFields); + } else { + throw new IllegalArgumentException("Can not get row from Array of type " + elementType); + } + } + + @Override + protected BinaryArray createNestedArrayInstance() { + if (elementType instanceof ArrayType) { + return new CompactedArray(((ArrayType) elementType).getElementType()); + } else { + throw new IllegalArgumentException( + "Can not get nested array from Array of type " + elementType); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/array/IndexedArray.java b/fluss-common/src/main/java/org/apache/fluss/row/array/IndexedArray.java new file mode 100644 index 0000000000..860da18fb1 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/array/IndexedArray.java @@ -0,0 +1,69 @@ +/* + * 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.fluss.row.array; + +import org.apache.fluss.row.BinaryArray; +import org.apache.fluss.row.BinarySegmentUtils; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.ArrayType; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; + +/** + * A {@link BinaryArray} that uses {@link org.apache.fluss.row.indexed.IndexedRow} as the binary + * format for arrays of nested row type. + */ +public class IndexedArray extends BinaryArray { + + private final DataType elementType; + + private transient DataType[] nestedFields; + + public IndexedArray(DataType elementType) { + this.elementType = elementType; + } + + @Override + public InternalRow getRow(int pos, int numFields) { + assertIndexIsValid(pos); + if (elementType instanceof RowType) { + if (nestedFields == null) { + nestedFields = ((RowType) elementType).getFieldTypes().toArray(new DataType[0]); + } + if (nestedFields.length != numFields) { + throw new IllegalArgumentException( + "Unexpected number of fields " + numFields + " for " + elementType); + } + return BinarySegmentUtils.readIndexedRow(segments, offset, getLong(pos), nestedFields); + } else { + throw new IllegalArgumentException("Can not get row from Array of type " + elementType); + } + } + + @Override + protected BinaryArray createNestedArrayInstance() { + if (elementType instanceof ArrayType) { + return new IndexedArray(((ArrayType) elementType).getElementType()); + } else { + throw new IllegalArgumentException( + "Can not get nested array from Array of type " + elementType); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/array/PrimitiveBinaryArray.java b/fluss-common/src/main/java/org/apache/fluss/row/array/PrimitiveBinaryArray.java new file mode 100644 index 0000000000..d91a0594da --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/array/PrimitiveBinaryArray.java @@ -0,0 +1,43 @@ +/* + * 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.fluss.row.array; + +import org.apache.fluss.row.BinaryArray; +import org.apache.fluss.row.InternalRow; + +/** + * A BinaryArray implementation for primitive types (except complex types) which does not support + * getRow operation. + */ +public class PrimitiveBinaryArray extends BinaryArray { + private static final long serialVersionUID = 1L; + + @Override + public InternalRow getRow(int pos, int numFields) { + throw new IllegalArgumentException("Can not get nested row from array of primitive type."); + } + + @Override + protected BinaryArray createNestedArrayInstance() { + // this should never be called from a primitive array, + // however, we still return a placeholder + return new PrimitiveBinaryArray(); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowRowColumnVector.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowRowColumnVector.java new file mode 100644 index 0000000000..0c325a53fa --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/vectors/ArrowRowColumnVector.java @@ -0,0 +1,46 @@ +/* + * 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.fluss.row.arrow.vectors; + +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.columnar.ColumnarRow; +import org.apache.fluss.row.columnar.RowColumnVector; +import org.apache.fluss.row.columnar.VectorizedColumnBatch; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; + +/** ArrowRowColumnVector is a wrapper class for Arrow RowVector. */ +public class ArrowRowColumnVector implements RowColumnVector { + private final StructVector vector; + private final VectorizedColumnBatch vectorizedColumnBatch; + + public ArrowRowColumnVector(StructVector vector, VectorizedColumnBatch columnBatch) { + this.vector = vector; + this.vectorizedColumnBatch = columnBatch; + } + + @Override + public InternalRow getRow(int i) { + return new ColumnarRow(vectorizedColumnBatch, i); + } + + @Override + public boolean isNullAt(int i) { + return vector.isNull(i); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowRowWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowRowWriter.java new file mode 100644 index 0000000000..59ee49bafd --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/arrow/writers/ArrowRowWriter.java @@ -0,0 +1,46 @@ +/* + * 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.fluss.row.arrow.writers; + +import org.apache.fluss.row.DataGetters; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.FieldVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; + +/** {@link ArrowFieldWriter} for Row. */ +public class ArrowRowWriter extends ArrowFieldWriter { + + private final ArrowFieldWriter[] fieldWriters; + + public ArrowRowWriter(FieldVector fieldVector, ArrowFieldWriter[] fieldWriters) { + super(fieldVector); + this.fieldWriters = fieldWriters; + } + + @Override + public void doWrite(int rowIndex, DataGetters row, int ordinal, boolean handleSafe) { + InternalRow nestedRow = row.getRow(ordinal, fieldWriters.length); + StructVector structVector = (StructVector) fieldVector; + + structVector.setIndexDefined(rowIndex); + for (int i = 0; i < fieldWriters.length; i++) { + fieldWriters[i].write(rowIndex, nestedRow, i, handleSafe); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java index 1a3639ea40..68b3d19fb9 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarArray.java @@ -21,6 +21,7 @@ import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; @@ -137,7 +138,11 @@ public InternalArray getArray(int pos) { } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + return ((RowColumnVector) data).getRow(offset + pos); + } @Override public boolean[] toBooleanArray() { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java index 8b03ca1d76..da0179d95c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/ColumnarRow.java @@ -138,7 +138,11 @@ public InternalArray getArray(int pos) { } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + return vectorizedColumnBatch.getRow(rowId, pos); + } @Override public int getFieldCount() { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/RowColumnVector.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/RowColumnVector.java new file mode 100644 index 0000000000..5ebf0a04df --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/RowColumnVector.java @@ -0,0 +1,39 @@ +/* + * 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.fluss.row.columnar; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.row.InternalRow; + +/** + * Row {@link ColumnVector}. + * + * @since 0.9 + */ +@Internal +public interface RowColumnVector extends ColumnVector { + + /** + * Return row value. + * + * @param index element index + * @return InternalRow + */ + InternalRow getRow(int index); +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java b/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java index d14b894453..d23a24d43c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/columnar/VectorizedColumnBatch.java @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; @@ -108,6 +109,7 @@ public InternalArray getArray(int rowId, int colId) { return ((ArrayColumnVector) columns[colId]).getArray(rowId); } - // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + public InternalRow getRow(int rowId, int colId) { + return ((RowColumnVector) columns[colId]).getRow(rowId); + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedKeyWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedKeyWriter.java index ba3e088c42..d960785e02 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedKeyWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedKeyWriter.java @@ -20,6 +20,8 @@ import org.apache.fluss.row.BinaryWriter; import org.apache.fluss.types.DataType; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.COMPACTED; + /** * A wrapping of {@link CompactedRowWriter} used to encode key columns. * @@ -35,7 +37,7 @@ public CompactedKeyWriter() { } public static ValueWriter createValueWriter(DataType fieldType) { - ValueWriter valueWriter = BinaryWriter.createValueWriter(fieldType); + ValueWriter valueWriter = BinaryWriter.createValueWriter(fieldType, COMPACTED); return (writer, pos, value) -> { if (value == null) { throw new IllegalArgumentException( diff --git a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java index a6b6f86c86..b6d0d3ecfb 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRow.java @@ -256,7 +256,11 @@ public InternalArray getArray(int pos) { } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + return decodedRow().getRow(pos, numFields); + } @Override public boolean equals(Object o) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java index a8e8058208..ad5d1b9e0c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowReader.java @@ -25,7 +25,10 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.array.CompactedArray; +import org.apache.fluss.types.ArrayType; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; import java.io.Serializable; @@ -319,17 +322,15 @@ static FieldReader createFieldReader(DataType fieldType) { fieldReader = (reader, pos) -> reader.readTimestampLtz(timestampLtzPrecision); break; case ARRAY: - fieldReader = (reader, pos) -> reader.readArray(); + DataType elementType = ((ArrayType) fieldType).getElementType(); + fieldReader = (reader, pos) -> reader.readArray(elementType); break; - case MAP: - // TODO: Map type support will be added in Issue #1973 - throw new UnsupportedOperationException( - "Map type in KV table is not supported yet. Will be added in Issue #1976."); case ROW: - // TODO: Row type support will be added in Issue #1974 - throw new UnsupportedOperationException( - "Row type in KV table is not supported yet. Will be added in Issue #1977."); + DataType[] nestedFieldTypes = + ((RowType) fieldType).getFieldTypes().toArray(new DataType[0]); + fieldReader = (reader, pos) -> reader.readRow(nestedFieldTypes); + break; default: throw new IllegalArgumentException("Unsupported type for IndexedRow: " + fieldType); } @@ -344,13 +345,23 @@ static FieldReader createFieldReader(DataType fieldType) { }; } - public InternalArray readArray() { + public InternalArray readArray(DataType elementType) { int length = readInt(); - InternalArray array = BinarySegmentUtils.readBinaryArray(segments, position, length); + InternalArray array = + BinarySegmentUtils.readBinaryArray( + segments, position, length, new CompactedArray(elementType)); position += length; return array; } + public InternalRow readRow(DataType[] nestedFieldTypes) { + int length = readInt(); + CompactedRow row = new CompactedRow(nestedFieldTypes); + row.pointTo(segments, position, length); + position += length; + return row; + } + /** * Accessor for reading the field of a row during runtime. * diff --git a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowWriter.java index e787d78621..2247344bd7 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/compacted/CompactedRowWriter.java @@ -22,14 +22,17 @@ import org.apache.fluss.memory.MemorySegmentWritable; import org.apache.fluss.memory.OutputView; import org.apache.fluss.row.BinaryArray; +import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.BinarySegmentUtils; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.SequentialBinaryWriter; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.row.serializer.ArraySerializer; +import org.apache.fluss.row.serializer.RowSerializer; import org.apache.fluss.utils.UnsafeUtils; import java.io.IOException; @@ -319,7 +322,7 @@ public void writeArray(InternalArray value, ArraySerializer serializer) { write(length, segments, offset); } - // TODO: Map and Row write methods will be added in Issue #1973 and #1974 + // TODO: Map and Row write methods will be added in Issue #1973 // public void writeMap(InternalMap value, InternalMapSerializer serializer) { // BinaryMap binaryMap = serializer.toBinaryMap(value); // MemorySegment[] segments = binaryMap.getSegments(); @@ -328,15 +331,16 @@ public void writeArray(InternalArray value, ArraySerializer serializer) { // // write(length, segments, offset); // } - // - // public void writeRow(InternalRow value, InternalRowSerializer serializer) { - // BinaryRow binaryRow = serializer.toBinaryRow(value); - // MemorySegment[] segments = binaryRow.getSegments(); - // int offset = binaryRow.getOffset(); - // int length = binaryRow.getSizeInBytes(); - // - // write(length, segments, offset); - // } + + @Override + public void writeRow(InternalRow value, RowSerializer serializer) { + BinaryRow binaryRow = serializer.toBinaryRow(value); + MemorySegment[] segments = binaryRow.getSegments(); + int offset = binaryRow.getOffset(); + int length = binaryRow.getSizeInBytes(); + + write(length, segments, offset); + } @Override public void complete() {} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/encode/AlignedRowEncoder.java b/fluss-common/src/main/java/org/apache/fluss/row/encode/AlignedRowEncoder.java new file mode 100644 index 0000000000..0bee6f4c36 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/encode/AlignedRowEncoder.java @@ -0,0 +1,69 @@ +/* + * 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.fluss.row.encode; + +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.BinaryWriter; +import org.apache.fluss.row.aligned.AlignedRow; +import org.apache.fluss.row.aligned.AlignedRowWriter; +import org.apache.fluss.types.DataType; + +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.ALIGNED; + +/** + * A {@link RowEncoder} for {@link AlignedRow}. + * + * @since 0.9 + */ +public class AlignedRowEncoder implements RowEncoder { + private final AlignedRow reuseRow; + private final AlignedRowWriter reuseWriter; + private final BinaryWriter.ValueWriter[] valueWriters; + + public AlignedRowEncoder(DataType[] fieldTypes) { + this.reuseRow = new AlignedRow(fieldTypes.length); + this.reuseWriter = new AlignedRowWriter(reuseRow); + this.valueWriters = new BinaryWriter.ValueWriter[fieldTypes.length]; + for (int i = 0; i < fieldTypes.length; i++) { + valueWriters[i] = BinaryWriter.createValueWriter(fieldTypes[i], ALIGNED); + } + } + + @Override + public void startNewRow() { + reuseWriter.reset(); + } + + @Override + public void encodeField(int pos, Object value) { + valueWriters[pos].writeValue(reuseWriter, pos, value); + } + + @Override + public BinaryRow finishRow() { + reuseWriter.complete(); + return reuseRow; + } + + @Override + public void close() throws Exception { + // nothing to close + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/encode/CompactedRowEncoder.java b/fluss-common/src/main/java/org/apache/fluss/row/encode/CompactedRowEncoder.java index 76c04f7773..546ce45a28 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/encode/CompactedRowEncoder.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/encode/CompactedRowEncoder.java @@ -24,6 +24,8 @@ import org.apache.fluss.row.compacted.CompactedRowWriter; import org.apache.fluss.types.DataType; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.COMPACTED; + /** * A {@link RowEncoder} for {@link CompactedRow}. * @@ -43,7 +45,7 @@ public CompactedRowEncoder(DataType[] fieldDataTypes) { writer = new CompactedRowWriter(fieldDataTypes.length); fieldWriters = new BinaryWriter.ValueWriter[fieldDataTypes.length]; for (int i = 0; i < fieldDataTypes.length; i++) { - fieldWriters[i] = BinaryWriter.createValueWriter(fieldDataTypes[i]); + fieldWriters[i] = BinaryWriter.createValueWriter(fieldDataTypes[i], COMPACTED); } this.compactedRowDeserializer = new CompactedRowDeserializer(fieldDataTypes); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/encode/IndexedRowEncoder.java b/fluss-common/src/main/java/org/apache/fluss/row/encode/IndexedRowEncoder.java index 0bb552e5af..53fc9fac14 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/encode/IndexedRowEncoder.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/encode/IndexedRowEncoder.java @@ -24,6 +24,8 @@ import org.apache.fluss.types.DataType; import org.apache.fluss.types.RowType; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.INDEXED; + /** * A {@link RowEncoder} for {@link IndexedRow}. * @@ -46,7 +48,7 @@ public IndexedRowEncoder(DataType[] fieldDataTypes) { this.fieldWriters = new BinaryWriter.ValueWriter[fieldDataTypes.length]; this.rowWriter = new IndexedRowWriter(fieldDataTypes); for (int i = 0; i < fieldDataTypes.length; i++) { - fieldWriters[i] = BinaryWriter.createValueWriter(fieldDataTypes[i]); + fieldWriters[i] = BinaryWriter.createValueWriter(fieldDataTypes[i], INDEXED); } } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java index bd9cdb99a8..a42fe16395 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRow.java @@ -30,11 +30,14 @@ import org.apache.fluss.row.NullAwareGetters; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.array.IndexedArray; +import org.apache.fluss.types.ArrayType; import org.apache.fluss.types.BinaryType; import org.apache.fluss.types.CharType; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DecimalType; import org.apache.fluss.types.IntType; +import org.apache.fluss.types.RowType; import org.apache.fluss.types.StringType; import org.apache.fluss.utils.MurmurHashUtils; @@ -236,7 +239,7 @@ public IndexedRow projectRow(int[] fields) { BinaryWriter.ValueWriter[] writers = new BinaryWriter.ValueWriter[newType.length]; for (int i = 0; i < newType.length; i++) { fieldGetter[i] = InternalRow.createFieldGetter(newType[i], fields[i]); - writers[i] = BinaryWriter.createValueWriter(newType[i]); + writers[i] = BinaryWriter.createValueWriter(newType[i], BinaryRowFormat.INDEXED); } IndexedRow projectRow = new IndexedRow(newType); @@ -383,11 +386,36 @@ public InternalArray getArray(int pos) { int offset = getFieldOffset(pos); int length = columnLengths[pos]; long offsetAndLength = ((long) offset << 32) | length; - return BinarySegmentUtils.readBinaryArray(segments, 0, offsetAndLength); + DataType fieldType = fieldTypes[pos]; + if (fieldType instanceof ArrayType) { + DataType elementType = ((ArrayType) fieldType).getElementType(); + return BinarySegmentUtils.readBinaryArray( + segments, 0, offsetAndLength, new IndexedArray(elementType)); + } else { + throw new IllegalStateException( + "Field type at position " + pos + " is not ArrayType: " + fieldType); + } } // TODO: getMap() will be added in Issue #1973 - // TODO: getRow() will be added in Issue #1974 + + @Override + public InternalRow getRow(int pos, int numFields) { + assertIndexIsValid(pos); + int offset = getFieldOffset(pos); + int length = columnLengths[pos]; + long offsetAndLength = ((long) offset << 32) | length; + DataType fieldType = fieldTypes[pos]; + if (fieldType instanceof RowType) { + DataType[] nestedFieldTypes = + ((RowType) fieldType).getFieldTypes().toArray(new DataType[0]); + return BinarySegmentUtils.readIndexedRow( + segments, 0, offsetAndLength, nestedFieldTypes); + } else { + throw new IllegalStateException( + "Field type at position " + pos + " is not RowType: " + fieldType); + } + } private void assertIndexIsValid(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; diff --git a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowReader.java b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowReader.java index b889ea7f2e..43f70cbfb7 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowReader.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowReader.java @@ -23,9 +23,13 @@ import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.array.IndexedArray; +import org.apache.fluss.types.ArrayType; import org.apache.fluss.types.DataType; +import org.apache.fluss.types.RowType; import java.io.Serializable; import java.util.Arrays; @@ -200,14 +204,25 @@ private byte[] readBytesInternal(int length) { return Arrays.copyOfRange(bytes, 0, newLen); } - public InternalArray readArray() { + public InternalArray readArray(DataType elementType) { int length = readVarLengthFromVarLengthList(); MemorySegment[] segments = new MemorySegment[] {segment}; - InternalArray array = BinarySegmentUtils.readBinaryArray(segments, position, length); + InternalArray array = + BinarySegmentUtils.readBinaryArray( + segments, position, length, new IndexedArray(elementType)); position += length; return array; } + public InternalRow readRow(DataType[] nestedFieldTypes) { + int length = readVarLengthFromVarLengthList(); + MemorySegment[] segments = new MemorySegment[] {segment}; + InternalRow row = + BinarySegmentUtils.readIndexedRow(segments, position, length, nestedFieldTypes); + position += length; + return row; + } + /** * Creates an accessor for reading elements. * @@ -268,16 +283,18 @@ static FieldReader createFieldReader(DataType fieldType) { fieldReader = (reader, pos) -> reader.readTimestampLtz(timestampLtzPrecision); break; case ARRAY: - fieldReader = (reader, pos) -> reader.readArray(); + DataType elementType = ((ArrayType) fieldType).getElementType(); + fieldReader = (reader, pos) -> reader.readArray(elementType); + break; + case ROW: + DataType[] nestedFieldTypes = + ((RowType) fieldType).getFieldTypes().toArray(new DataType[0]); + fieldReader = (reader, pos) -> reader.readRow(nestedFieldTypes); break; case MAP: // TODO: Map type support will be added in Issue #1973 throw new UnsupportedOperationException( "Map type for Indexed row format is not supported yet."); - case ROW: - // TODO: Row type support will be added in Issue #1974 - throw new UnsupportedOperationException( - "Row type for Indexed row format is not supported yet."); default: throw new IllegalArgumentException("Unsupported type for IndexedRow: " + fieldType); } diff --git a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowWriter.java b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowWriter.java index 0c956e343d..12fb2f0902 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowWriter.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/indexed/IndexedRowWriter.java @@ -22,14 +22,17 @@ import org.apache.fluss.memory.MemorySegmentWritable; import org.apache.fluss.memory.OutputView; import org.apache.fluss.row.BinaryArray; +import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.BinarySegmentUtils; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.SequentialBinaryWriter; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.row.serializer.ArraySerializer; +import org.apache.fluss.row.serializer.RowSerializer; import org.apache.fluss.types.DataType; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.UnsafeUtils; @@ -225,15 +228,16 @@ public void writeArray(InternalArray value, ArraySerializer serializer) { // // write(length, segments, offset); // } - // - // public void writeRow(InternalRow value, InternalRowSerializer serializer) { - // BinaryRow binaryRow = serializer.toBinaryRow(value); - // MemorySegment[] segments = binaryRow.getSegments(); - // int offset = binaryRow.getOffset(); - // int length = binaryRow.getSizeInBytes(); - // - // write(length, segments, offset); - // } + + @Override + public void writeRow(InternalRow value, RowSerializer serializer) { + BinaryRow binaryRow = serializer.toBinaryRow(value); + MemorySegment[] segments = binaryRow.getSegments(); + int offset = binaryRow.getOffset(); + int length = binaryRow.getSizeInBytes(); + + write(length, segments, offset); + } @Override public void complete() {} diff --git a/fluss-common/src/main/java/org/apache/fluss/row/serializer/ArraySerializer.java b/fluss-common/src/main/java/org/apache/fluss/row/serializer/ArraySerializer.java index ecfdcab0dc..004a3c6e4a 100644 --- a/fluss-common/src/main/java/org/apache/fluss/row/serializer/ArraySerializer.java +++ b/fluss-common/src/main/java/org/apache/fluss/row/serializer/ArraySerializer.java @@ -19,23 +19,34 @@ import org.apache.fluss.row.BinaryArray; import org.apache.fluss.row.BinaryArrayWriter; +import org.apache.fluss.row.BinaryRow.BinaryRowFormat; import org.apache.fluss.row.BinaryWriter; import org.apache.fluss.row.GenericArray; import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.array.AlignedArray; +import org.apache.fluss.row.array.CompactedArray; +import org.apache.fluss.row.array.IndexedArray; +import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.types.DataType; import java.io.Serializable; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.ALIGNED; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.COMPACTED; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.INDEXED; + /** Serializer for {@link InternalArray} to {@link BinaryArray} and {@code CompactedArray}. */ public class ArraySerializer implements Serializable { private static final long serialVersionUID = 1L; private final DataType eleType; + private final BinaryRowFormat rowFormat; private transient BinaryArraySerializer alignedSerializer; - public ArraySerializer(DataType eleType) { + public ArraySerializer(DataType eleType, BinaryRowFormat rowFormat) { this.eleType = eleType; + this.rowFormat = rowFormat; } public BinaryArray toBinaryArray(InternalArray from) { @@ -45,6 +56,19 @@ public BinaryArray toBinaryArray(InternalArray from) { return alignedSerializer.toAlignedArray(from); } + private BinaryArray createBinaryArrayInstance() { + switch (rowFormat) { + case COMPACTED: + return new CompactedArray(eleType); + case INDEXED: + return new IndexedArray(eleType); + case ALIGNED: + return new AlignedArray(); + default: + throw new IllegalArgumentException("Unsupported row format: " + rowFormat); + } + } + // ------------------------------------------------------------------------------------------ /** Serializer function for AlignedArray. */ @@ -57,7 +81,13 @@ private class BinaryArraySerializer { public BinaryArray toAlignedArray(InternalArray from) { if (from instanceof BinaryArray) { - return (BinaryArray) from; + if (from instanceof PrimitiveBinaryArray + || rowFormat == INDEXED && from instanceof IndexedArray + || rowFormat == COMPACTED && from instanceof CompactedArray + || rowFormat == ALIGNED && from instanceof AlignedArray) { + // directly return the original array iff the array is in the expected format + return (BinaryArray) from; + } } if (from instanceof GenericArray) { @@ -86,7 +116,7 @@ public BinaryArray toAlignedArray(InternalArray from) { int numElements = from.size(); if (reuseArray == null) { - reuseArray = new BinaryArray(); + reuseArray = createBinaryArrayInstance(); } if (reuseWriter == null || reuseWriter.getNumElements() != numElements) { reuseWriter = @@ -101,7 +131,7 @@ public BinaryArray toAlignedArray(InternalArray from) { elementGetter = InternalArray.createElementGetter(eleType); } if (valueWriter == null) { - valueWriter = BinaryWriter.createValueWriter(eleType); + valueWriter = BinaryWriter.createValueWriter(eleType, rowFormat); } for (int i = 0; i < numElements; i++) { diff --git a/fluss-common/src/main/java/org/apache/fluss/row/serializer/RowSerializer.java b/fluss-common/src/main/java/org/apache/fluss/row/serializer/RowSerializer.java new file mode 100644 index 0000000000..3a6c11dfb1 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/row/serializer/RowSerializer.java @@ -0,0 +1,113 @@ +/* + * 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.fluss.row.serializer; + +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.BinaryRow.BinaryRowFormat; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.aligned.AlignedRow; +import org.apache.fluss.row.compacted.CompactedRow; +import org.apache.fluss.row.encode.AlignedRowEncoder; +import org.apache.fluss.row.encode.CompactedRowEncoder; +import org.apache.fluss.row.encode.IndexedRowEncoder; +import org.apache.fluss.row.encode.RowEncoder; +import org.apache.fluss.row.indexed.IndexedRow; +import org.apache.fluss.types.DataType; + +import java.io.Serializable; + +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.ALIGNED; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.COMPACTED; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.INDEXED; + +/** Serializer for {@link InternalRow} to {@link BinaryRow}. */ +public class RowSerializer implements Serializable { + private static final long serialVersionUID = 1L; + + private final DataType[] fieldTypes; + private final BinaryRowFormat format; + + private transient BinaryRowSerializer serializer; + + public RowSerializer(DataType[] fieldTypes, BinaryRowFormat format) { + this.fieldTypes = fieldTypes; + this.format = format; + } + + /** + * Serialize the given {@link InternalRow} into {@link BinaryRow}. + * + *

The returned {@link BinaryRow} might reuse the memory from the input {@link InternalRow} + * if it is already a {@link BinaryRow}. + * + *

Otherwise, it will serialize a {@link BinaryRow} based on the specified {@link + * BinaryRowFormat}. + */ + public BinaryRow toBinaryRow(InternalRow from) { + if (from instanceof BinaryRow) { + if (format == INDEXED && from instanceof IndexedRow + || format == COMPACTED && from instanceof CompactedRow + || format == ALIGNED && from instanceof AlignedRow) { + // directly return the original row iff the row is in the expected format + return (BinaryRow) from; + } + } + + if (serializer == null) { + serializer = new BinaryRowSerializer(fieldTypes, format); + } + return serializer.toBinaryRow(from); + } + + /** + * Serializer function for BinaryRow, it delegates the actual encoding to different {@link + * RowEncoder} based on the specified format. + */ + private static class BinaryRowSerializer { + private final RowEncoder rowEncoder; + private final InternalRow.FieldGetter[] fieldGetters; + + private BinaryRowSerializer(DataType[] fieldTypes, BinaryRowFormat format) { + this.fieldGetters = new InternalRow.FieldGetter[fieldTypes.length]; + for (int i = 0; i < fieldTypes.length; i++) { + this.fieldGetters[i] = InternalRow.createFieldGetter(fieldTypes[i], i); + } + switch (format) { + case COMPACTED: + this.rowEncoder = new CompactedRowEncoder(fieldTypes); + break; + case INDEXED: + this.rowEncoder = new IndexedRowEncoder(fieldTypes); + break; + case ALIGNED: + this.rowEncoder = new AlignedRowEncoder(fieldTypes); + break; + default: + throw new IllegalArgumentException("Unsupported binary row format: " + format); + } + } + + public BinaryRow toBinaryRow(InternalRow from) { + rowEncoder.startNewRow(); + for (int i = 0; i < fieldGetters.length; i++) { + rowEncoder.encodeField(i, fieldGetters[i].getFieldOrNull(from)); + } + return rowEncoder.finishRow(); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/IntObjectHashMap.java b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/IntObjectHashMap.java new file mode 100644 index 0000000000..c9ecf04d13 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/IntObjectHashMap.java @@ -0,0 +1,730 @@ +/* + * 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.fluss.shaded.arrow.org.apache.arrow.vector.util; + +import java.util.AbstractCollection; +import java.util.AbstractSet; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; + +// TODO: This is a temporary workaround to resolve Arrow 15's dependency on eclipse-collections, +// which carries a Category-B license (see https://github.com/apache/arrow/issues/40896). +// Without these override classes, ArrowReaderWriterTest fails with a ClassNotFoundException. +// This workaround should be removed once we upgrade to Arrow 16 or later. Alternatively, these +// override classes could be moved to `fluss-shaded-arrow` to isolate the dependency. + +/** + * A vendored specialized copy of Netty's IntObjectHashMap for use within Arrow. Avoids requiring + * Netty in the Arrow core just for this one class. + * + * @param The value type stored in the map. + */ +class IntObjectHashMap implements IntObjectMap { + + /** Default initial capacity. Used if not specified in the constructor */ + public static final int DEFAULT_CAPACITY = 8; + + /** Default load factor. Used if not specified in the constructor */ + public static final float DEFAULT_LOAD_FACTOR = 0.5f; + + /** + * Placeholder for null values, so we can use the actual null to mean available. (Better than + * using a placeholder for available: less references for GC processing.) + */ + private static final Object NULL_VALUE = new Object(); + + /** The maximum number of elements allowed without allocating more space. */ + private int maxSize; + + /** The load factor for the map. Used to calculate {@link #maxSize}. */ + private final float loadFactor; + + private int[] keys; + private V[] values; + private int size; + private int mask; + + private final Set keySet = new KeySet(); + private final Set> entrySet = new EntrySet(); + private final Iterable> entries = + new Iterable>() { + @Override + public Iterator> iterator() { + return new PrimitiveIterator(); + } + }; + + public IntObjectHashMap() { + this(DEFAULT_CAPACITY, DEFAULT_LOAD_FACTOR); + } + + public IntObjectHashMap(int initialCapacity) { + this(initialCapacity, DEFAULT_LOAD_FACTOR); + } + + public IntObjectHashMap(int initialCapacity, float loadFactor) { + if (loadFactor <= 0.0f || loadFactor > 1.0f) { + // Cannot exceed 1 because we can never store more than capacity elements; + // using a bigger loadFactor would trigger rehashing before the desired load is reached. + throw new IllegalArgumentException("loadFactor must be > 0 and <= 1"); + } + + this.loadFactor = loadFactor; + + // Adjust the initial capacity if necessary. + int capacity = safeFindNextPositivePowerOfTwo(initialCapacity); + mask = capacity - 1; + + // Allocate the arrays. + keys = new int[capacity]; + @SuppressWarnings({"unchecked", "SuspiciousArrayCast"}) + V[] temp = (V[]) new Object[capacity]; + values = temp; + + // Initialize the maximum size value. + maxSize = calcMaxSize(capacity); + } + + private static T toExternal(T value) { + assert value != null : "null is not a legitimate internal value. Concurrent Modification?"; + return value == NULL_VALUE ? null : value; + } + + @SuppressWarnings("unchecked") + private static T toInternal(T value) { + return value == null ? (T) NULL_VALUE : value; + } + + @Override + public V get(int key) { + int index = indexOf(key); + return index == -1 ? null : toExternal(values[index]); + } + + @Override + public V put(int key, V value) { + int startIndex = hashIndex(key); + int index = startIndex; + + for (; ; ) { + if (values[index] == null) { + // Found empty slot, use it. + keys[index] = key; + values[index] = toInternal(value); + growSize(); + return null; + } + if (keys[index] == key) { + // Found existing entry with this key, just replace the value. + V previousValue = values[index]; + values[index] = toInternal(value); + return toExternal(previousValue); + } + + // Conflict, keep probing ... + if ((index = probeNext(index)) == startIndex) { + // Can only happen if the map was full at MAX_ARRAY_SIZE and couldn't grow. + throw new IllegalStateException("Unable to insert"); + } + } + } + + @Override + public void putAll(Map sourceMap) { + if (sourceMap instanceof IntObjectHashMap) { + // Optimization - iterate through the arrays. + @SuppressWarnings("unchecked") + IntObjectHashMap source = (IntObjectHashMap) sourceMap; + for (int i = 0; i < source.values.length; ++i) { + V sourceValue = source.values[i]; + if (sourceValue != null) { + put(source.keys[i], sourceValue); + } + } + return; + } + + // Otherwise, just add each entry. + for (Entry entry : sourceMap.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + @Override + public V remove(int key) { + int index = indexOf(key); + if (index == -1) { + return null; + } + + V prev = values[index]; + removeAt(index); + return toExternal(prev); + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return size == 0; + } + + @Override + public void clear() { + Arrays.fill(keys, (int) 0); + Arrays.fill(values, null); + size = 0; + } + + @Override + public boolean containsKey(int key) { + return indexOf(key) >= 0; + } + + @Override + public boolean containsValue(Object value) { + @SuppressWarnings("unchecked") + V v1 = toInternal((V) value); + for (V v2 : values) { + // The map supports null values; this will be matched as NULL_VALUE.equals(NULL_VALUE). + if (v2 != null && v2.equals(v1)) { + return true; + } + } + return false; + } + + @Override + public Iterable> entries() { + return entries; + } + + @Override + public Collection values() { + return new AbstractCollection() { + @Override + public Iterator iterator() { + return new Iterator() { + final PrimitiveIterator iter = new PrimitiveIterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public V next() { + return iter.next().value(); + } + + @Override + public void remove() { + iter.remove(); + } + }; + } + + @Override + public int size() { + return size; + } + }; + } + + @Override + public int hashCode() { + // Hashcode is based on all non-zero, valid keys. We have to scan the whole keys + // array, which may have different lengths for two maps of same size(), so the + // capacity cannot be used as input for hashing but the size can. + int hash = size; + for (int key : keys) { + // 0 can be a valid key or unused slot, but won't impact the hashcode in either case. + // This way we can use a cheap loop without conditionals, or hard-to-unroll operations, + // or the devastatingly bad memory locality of visiting value objects. + // Also, it's important to use a hash function that does not depend on the ordering + // of terms, only their values; since the map is an unordered collection and + // entries can end up in different positions in different maps that have the same + // elements, but with different history of puts/removes, due to conflicts. + hash ^= hashCode(key); + } + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof IntObjectMap)) { + return false; + } + @SuppressWarnings("rawtypes") + IntObjectMap other = (IntObjectMap) obj; + if (size != other.size()) { + return false; + } + for (int i = 0; i < values.length; ++i) { + V value = values[i]; + if (value != null) { + int key = keys[i]; + Object otherValue = other.get(key); + if (value == NULL_VALUE) { + if (otherValue != null) { + return false; + } + } else if (!value.equals(otherValue)) { + return false; + } + } + } + return true; + } + + @Override + public boolean containsKey(Object key) { + return containsKey(objectToKey(key)); + } + + @Override + public V get(Object key) { + return get(objectToKey(key)); + } + + @Override + public V put(Integer key, V value) { + return put(objectToKey(key), value); + } + + @Override + public V remove(Object key) { + return remove(objectToKey(key)); + } + + @Override + public Set keySet() { + return keySet; + } + + @Override + public Set> entrySet() { + return entrySet; + } + + private int objectToKey(Object key) { + return (int) (Integer) key; + } + + /** + * Locates the index for the given key. This method probes using double hashing. + * + * @param key the key for an entry in the map. + * @return the index where the key was found, or {@code -1} if no entry is found for that key. + */ + private int indexOf(int key) { + int startIndex = hashIndex(key); + int index = startIndex; + + for (; ; ) { + if (values[index] == null) { + // It's available, so no chance that this value exists anywhere in the map. + return -1; + } + if (key == keys[index]) { + return index; + } + + // Conflict, keep probing ... + if ((index = probeNext(index)) == startIndex) { + return -1; + } + } + } + + /** Returns the hashed index for the given key. */ + private int hashIndex(int key) { + // The array lengths are always a power of two, so we can use a bitmask to stay inside the + // array bounds. + return hashCode(key) & mask; + } + + /** Returns the hash code for the key. */ + private static int hashCode(int key) { + return key; + } + + /** Get the next sequential index after {@code index} and wraps if necessary. */ + private int probeNext(int index) { + // The array lengths are always a power of two, so we can use a bitmask to stay inside the + // array bounds. + return (index + 1) & mask; + } + + /** Grows the map size after an insertion. If necessary, performs a rehash of the map. */ + private void growSize() { + size++; + + if (size > maxSize) { + if (keys.length == Integer.MAX_VALUE) { + throw new IllegalStateException("Max capacity reached at size=" + size); + } + + // Double the capacity. + rehash(keys.length << 1); + } + } + + /** + * Removes entry at the given index position. Also performs opportunistic, incremental rehashing + * if necessary to not break conflict chains. + * + * @param index the index position of the element to remove. + * @return {@code true} if the next item was moved back. {@code false} otherwise. + */ + private boolean removeAt(final int index) { + --size; + // Clearing the key is not strictly necessary (for GC like in a regular collection), + // but recommended for security. The memory location is still fresh in the cache anyway. + keys[index] = 0; + values[index] = null; + + // In the interval from index to the next available entry, the arrays may have entries + // that are displaced from their base position due to prior conflicts. Iterate these + // entries and move them back if possible, optimizing future lookups. + // Knuth Section 6.4 Algorithm R, also used by the JDK's IdentityHashMap. + + int nextFree = index; + int i = probeNext(index); + for (V value = values[i]; value != null; value = values[i = probeNext(i)]) { + int key = keys[i]; + int bucket = hashIndex(key); + if (i < bucket && (bucket <= nextFree || nextFree <= i) + || bucket <= nextFree && nextFree <= i) { + // Move the displaced entry "back" to the first available position. + keys[nextFree] = key; + values[nextFree] = value; + // Put the first entry after the displaced entry + keys[i] = 0; + values[i] = null; + nextFree = i; + } + } + return nextFree != index; + } + + /** Calculates the maximum size allowed before rehashing. */ + private int calcMaxSize(int capacity) { + // Clip the upper bound so that there will always be at least one available slot. + int upperBound = capacity - 1; + return Math.min(upperBound, (int) (capacity * loadFactor)); + } + + /** + * Rehashes the map for the given capacity. + * + * @param newCapacity the new capacity for the map. + */ + private void rehash(int newCapacity) { + int[] oldKeys = keys; + V[] oldVals = values; + + keys = new int[newCapacity]; + @SuppressWarnings({"unchecked", "SuspiciousArrayCast"}) + V[] temp = (V[]) new Object[newCapacity]; + values = temp; + + maxSize = calcMaxSize(newCapacity); + mask = newCapacity - 1; + + // Insert to the new arrays. + for (int i = 0; i < oldVals.length; ++i) { + V oldVal = oldVals[i]; + if (oldVal != null) { + // Inlined put(), but much simpler: we don't need to worry about + // duplicated keys, growing/rehashing, or failing to insert. + int oldKey = oldKeys[i]; + int index = hashIndex(oldKey); + + for (; ; ) { + if (values[index] == null) { + keys[index] = oldKey; + values[index] = oldVal; + break; + } + + // Conflict, keep probing. Can wrap around, but never reaches startIndex again. + index = probeNext(index); + } + } + } + } + + @Override + public String toString() { + if (isEmpty()) { + return "{}"; + } + StringBuilder sb = new StringBuilder(4 * size); + sb.append('{'); + boolean first = true; + for (int i = 0; i < values.length; ++i) { + V value = values[i]; + if (value != null) { + if (!first) { + sb.append(", "); + } + sb.append(keyToString(keys[i])) + .append('=') + .append(value == this ? "(this Map)" : toExternal(value)); + first = false; + } + } + return sb.append('}').toString(); + } + + /** + * Helper method called by {@link #toString()} in order to convert a single map key into a + * string. This is protected to allow subclasses to override the appearance of a given key. + */ + protected String keyToString(int key) { + return Integer.toString(key); + } + + /** Set implementation for iterating over the entries of the map. */ + private final class EntrySet extends AbstractSet> { + @Override + public Iterator> iterator() { + return new MapIterator(); + } + + @Override + public int size() { + return IntObjectHashMap.this.size(); + } + } + + /** Set implementation for iterating over the keys. */ + private final class KeySet extends AbstractSet { + @Override + public int size() { + return IntObjectHashMap.this.size(); + } + + @Override + public boolean contains(Object o) { + return IntObjectHashMap.this.containsKey(o); + } + + @Override + public boolean remove(Object o) { + return IntObjectHashMap.this.remove(o) != null; + } + + @Override + public boolean retainAll(Collection retainedKeys) { + boolean changed = false; + for (Iterator> iter = entries().iterator(); iter.hasNext(); ) { + PrimitiveEntry entry = iter.next(); + if (!retainedKeys.contains(entry.key())) { + changed = true; + iter.remove(); + } + } + return changed; + } + + @Override + public void clear() { + IntObjectHashMap.this.clear(); + } + + @Override + public Iterator iterator() { + return new Iterator() { + private final Iterator> iter = entrySet.iterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Integer next() { + return iter.next().getKey(); + } + + @Override + public void remove() { + iter.remove(); + } + }; + } + } + + /** + * Iterator over primitive entries. Entry key/values are overwritten by each call to {@link + * #next()}. + */ + private final class PrimitiveIterator + implements Iterator>, PrimitiveEntry { + private int prevIndex = -1; + private int nextIndex = -1; + private int entryIndex = -1; + + private void scanNext() { + while (++nextIndex != values.length && values[nextIndex] == null) {} + } + + @Override + public boolean hasNext() { + if (nextIndex == -1) { + scanNext(); + } + return nextIndex != values.length; + } + + @Override + public PrimitiveEntry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + prevIndex = nextIndex; + scanNext(); + + // Always return the same Entry object, just change its index each time. + entryIndex = prevIndex; + return this; + } + + @Override + public void remove() { + if (prevIndex == -1) { + throw new IllegalStateException("next must be called before each remove."); + } + if (removeAt(prevIndex)) { + // removeAt may move elements "back" in the array if they have been displaced + // because their spot in the + // array was occupied when they were inserted. If this occurs then the nextIndex is + // now invalid and + // should instead point to the prevIndex which now holds an element which was "moved + // back". + nextIndex = prevIndex; + } + prevIndex = -1; + } + + // Entry implementation. Since this implementation uses a single Entry, we coalesce that + // into the Iterator object (potentially making loop optimization much easier). + + @Override + public int key() { + return keys[entryIndex]; + } + + @Override + public V value() { + return toExternal(values[entryIndex]); + } + + @Override + public void setValue(V value) { + values[entryIndex] = toInternal(value); + } + } + + /** Iterator used by the {@link Map} interface. */ + private final class MapIterator implements Iterator> { + private final PrimitiveIterator iter = new PrimitiveIterator(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + iter.next(); + + return new MapEntry(iter.entryIndex); + } + + @Override + public void remove() { + iter.remove(); + } + } + + /** A single entry in the map. */ + final class MapEntry implements Entry { + private final int entryIndex; + + MapEntry(int entryIndex) { + this.entryIndex = entryIndex; + } + + @Override + public Integer getKey() { + verifyExists(); + return keys[entryIndex]; + } + + @Override + public V getValue() { + verifyExists(); + return toExternal(values[entryIndex]); + } + + @Override + public V setValue(V value) { + verifyExists(); + V prevValue = toExternal(values[entryIndex]); + values[entryIndex] = toInternal(value); + return prevValue; + } + + private void verifyExists() { + if (values[entryIndex] == null) { + throw new IllegalStateException("The map entry has been removed"); + } + } + } + + static int safeFindNextPositivePowerOfTwo(final int value) { + return value <= 0 + ? 1 + : value >= 0x40000000 ? 0x40000000 : findNextPositivePowerOfTwo(value); + } + + static int findNextPositivePowerOfTwo(final int value) { + assert value > Integer.MIN_VALUE && value < 0x40000000; + return 1 << (32 - Integer.numberOfLeadingZeros(value - 1)); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/IntObjectMap.java b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/IntObjectMap.java new file mode 100644 index 0000000000..60f61787c0 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/IntObjectMap.java @@ -0,0 +1,89 @@ +/* + * 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.fluss.shaded.arrow.org.apache.arrow.vector.util; + +import java.util.Iterator; +import java.util.Map; + +// TODO: This is a temporary workaround to resolve Arrow 15's dependency on eclipse-collections, +// which carries a Category-B license (see https://github.com/apache/arrow/issues/40896). +// Without these override classes, ArrowReaderWriterTest fails with a ClassNotFoundException. +// This workaround should be removed once we upgrade to Arrow 16 or later. Alternatively, these +// override classes could be moved to `fluss-shaded-arrow` to isolate the dependency. +/** + * A vendored specialized copy of Netty's IntObjectMap for use within Arrow. Avoids requiring Netty + * in the Arrow core just for this one class. + * + * @param the value type stored in the map. + */ +interface IntObjectMap extends Map { + + /** + * A primitive entry in the map, provided by the iterator from {@link #entries()}. + * + * @param the value type stored in the map. + */ + interface PrimitiveEntry { + /** Gets the key for this entry. */ + int key(); + + /** Gets the value for this entry. */ + V value(); + + /** Sets the value for this entry. */ + void setValue(V value); + } + + /** + * Gets the value in the map with the specified key. + * + * @param key the key whose associated value is to be returned. + * @return the value or {@code null} if the key was not found in the map. + */ + V get(int key); + + /** + * Puts the given entry into the map. + * + * @param key the key of the entry. + * @param value the value of the entry. + * @return the previous value for this key or {@code null} if there was no previous mapping. + */ + V put(int key, V value); + + /** + * Removes the entry with the specified key. + * + * @param key the key for the entry to be removed from this map. + * @return the previous value for the key, or {@code null} if there was no mapping. + */ + V remove(int key); + + /** + * Gets an iterable to traverse over the primitive entries contained in this map. As an + * optimization, the {@link PrimitiveEntry}s returned by the {@link Iterator} may change as the + * {@link Iterator} progresses. The caller should not rely on {@link PrimitiveEntry} key/value + * stability. + */ + Iterable> entries(); + + /** Indicates whether or not this map contains a value for the specified key. */ + boolean containsKey(int key); +} diff --git a/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/MapWithOrdinalImpl.java b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/MapWithOrdinalImpl.java new file mode 100644 index 0000000000..528655d7bd --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/MapWithOrdinalImpl.java @@ -0,0 +1,248 @@ +/* + * 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.fluss.shaded.arrow.org.apache.arrow.vector.util; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +// TODO: This is a temporary workaround to resolve Arrow 15's dependency on eclipse-collections, +// which carries a Category-B license (see https://github.com/apache/arrow/issues/40896). +// Without these override classes, ArrowReaderWriterTest fails with a ClassNotFoundException. +// This workaround should be removed once we upgrade to Arrow 16 or later. Alternatively, these +// override classes could be moved to `fluss-shaded-arrow` to isolate the dependency. + +/** + * An implementation of map that supports constant time look-up by a generic key or an ordinal. + * + *

This class extends the functionality a regular {@link Map} with ordinal lookup support. Upon + * insertion an unused ordinal is assigned to the inserted (key, value) tuple. Upon update the same + * ordinal id is re-used while value is replaced. Upon deletion of an existing item, its + * corresponding ordinal is recycled and could be used by another item. + * + *

For any instance with N items, this implementation guarantees that ordinals are in the range + * of [0, N). However, the ordinal assignment is dynamic and may change after an insertion or + * deletion. Consumers of this class are responsible for explicitly checking the ordinal + * corresponding to a key via {@link MapWithOrdinalImpl#getOrdinal(Object)} before attempting to + * execute a lookup with an ordinal. + * + * @param key type + * @param value type + */ +public class MapWithOrdinalImpl implements MapWithOrdinal { + + private final Map> primary = new LinkedHashMap<>(); + private final IntObjectHashMap secondary = new IntObjectHashMap<>(); + + private final Map delegate = + new Map() { + @Override + public boolean isEmpty() { + return size() == 0; + } + + @Override + public int size() { + return primary.size(); + } + + @Override + public boolean containsKey(Object key) { + return primary.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return primary.containsValue(value); + } + + @Override + public V get(Object key) { + Entry pair = primary.get(key); + if (pair != null) { + return pair.getValue(); + } + return null; + } + + @Override + public V put(K key, V value) { + final Entry oldPair = primary.get(key); + // if key exists try replacing otherwise, assign a new ordinal identifier + final int ordinal = oldPair == null ? primary.size() : oldPair.getKey(); + primary.put(key, new AbstractMap.SimpleImmutableEntry<>(ordinal, value)); + secondary.put(ordinal, value); + return oldPair == null ? null : oldPair.getValue(); + } + + @Override + public V remove(Object key) { + final Entry oldPair = primary.remove(key); + if (oldPair != null) { + final int lastOrdinal = secondary.size(); + final V last = secondary.get(lastOrdinal); + // normalize mappings so that all numbers until primary.size() is assigned + // swap the last element with the deleted one + secondary.put(oldPair.getKey(), last); + primary.put( + (K) key, + new AbstractMap.SimpleImmutableEntry<>(oldPair.getKey(), last)); + } + return oldPair == null ? null : oldPair.getValue(); + } + + @Override + public void putAll(Map m) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + primary.clear(); + secondary.clear(); + } + + @Override + public Set keySet() { + return primary.keySet(); + } + + @Override + public Collection values() { + return secondary.values(); + } + + @Override + public Set> entrySet() { + return primary.entrySet().stream() + .map( + entry -> + new AbstractMap.SimpleImmutableEntry<>( + entry.getKey(), entry.getValue().getValue())) + .collect(Collectors.toSet()); + } + }; + + /** + * Returns the value corresponding to the given ordinal. + * + * @param id ordinal value for lookup + * @return an instance of V + */ + @Override + public V getByOrdinal(int id) { + return secondary.get(id); + } + + /** + * Returns the ordinal corresponding to the given key. + * + * @param key key for ordinal lookup + * @return ordinal value corresponding to key if it exists or -1 + */ + @Override + public int getOrdinal(K key) { + Map.Entry pair = primary.get(key); + if (pair != null) { + return pair.getKey(); + } + return -1; + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean isEmpty() { + return delegate.isEmpty(); + } + + @Override + public Collection getAll(K key) { + if (delegate.containsKey(key)) { + List list = new ArrayList<>(1); + list.add(get(key)); + return list; + } + return null; + } + + @Override + public V get(K key) { + return delegate.get(key); + } + + /** + * Inserts the tuple (key, value) into the map extending the semantics of {@link Map#put} with + * automatic ordinal assignment. A new ordinal is assigned if key does not exists. Otherwise the + * same ordinal is re-used but the value is replaced. + * + * @see java.util.Map#put + */ + @Override + public boolean put(K key, V value, boolean overwrite) { + return delegate.put(key, value) != null; + } + + @Override + public Collection values() { + return delegate.values(); + } + + @Override + public boolean remove(K key, V value) { + return false; + } + + @Override + public boolean containsKey(Object key) { + return delegate.containsKey(key); + } + + /** + * Removes the element corresponding to the key if exists extending the semantics of {@link + * java.util.Map#remove} with ordinal re-cycling. The ordinal corresponding to the given key may + * be re-assigned to another tuple. It is important that consumer checks the ordinal value via + * {@link MapWithOrdinalImpl#getOrdinal(Object)} before attempting to look-up by ordinal. + * + * @see java.util.Map#remove + */ + @Override + public boolean removeAll(K key) { + return delegate.remove(key) != null; + } + + @Override + public void clear() { + delegate.clear(); + } + + @Override + public Set keys() { + return delegate.keySet(); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/MultiMapWithOrdinal.java b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/MultiMapWithOrdinal.java new file mode 100644 index 0000000000..964a1eef9c --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/shaded/arrow/org/apache/arrow/vector/util/MultiMapWithOrdinal.java @@ -0,0 +1,239 @@ +/* + * 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.fluss.shaded.arrow.org.apache.arrow.vector.util; + +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +// TODO: This is a temporary workaround to resolve Arrow 15's dependency on eclipse-collections, +// which carries a Category-B license (see https://github.com/apache/arrow/issues/40896). +// Without these override classes, ArrowReaderWriterTest fails with a ClassNotFoundException. +// This workaround should be removed once we upgrade to Arrow 16 or later. Alternatively, these +// override classes could be moved to `fluss-shaded-arrow` to isolate the dependency. + +/** + * An implementation of a multimap that supports constant time look-up by a generic key or an + * ordinal. + * + *

This class extends the functionality a regular {@link Map} with ordinal lookup support. Upon + * insertion an unused ordinal is assigned to the inserted (key, value) tuple. Upon update the same + * ordinal id is re-used while value is replaced. Upon deletion of an existing item, its + * corresponding ordinal is recycled and could be used by another item. + * + *

For any instance with N items, this implementation guarantees that ordinals are in the range + * of [0, N). However, the ordinal assignment is dynamic and may change after an insertion or + * deletion. Consumers of this class are responsible for explicitly checking the ordinal + * corresponding to a key via {@link MultiMapWithOrdinal#getOrdinal(Object)} before attempting to + * execute a lookup with an ordinal. + * + * @param key type + * @param value type + */ +public class MultiMapWithOrdinal implements MapWithOrdinal { + + private final Map> keyToOrdinal = new LinkedHashMap<>(); + private final IntObjectHashMap ordinalToValue = new IntObjectHashMap<>(); + + /** + * Returns the value corresponding to the given ordinal. + * + * @param id ordinal value for lookup + * @return an instance of V + */ + @Override + public V getByOrdinal(int id) { + return ordinalToValue.get(id); + } + + /** + * Returns the ordinal corresponding to the given key. + * + * @param key key for ordinal lookup + * @return ordinal value corresponding to key if it exists or -1 + */ + @Override + public int getOrdinal(K key) { + Set pair = getOrdinals(key); + if (!pair.isEmpty()) { + return pair.iterator().next(); + } + return -1; + } + + private Set getOrdinals(K key) { + return keyToOrdinal.getOrDefault(key, new HashSet<>()); + } + + @Override + public int size() { + return ordinalToValue.size(); + } + + @Override + public boolean isEmpty() { + return ordinalToValue.isEmpty(); + } + + /** get set of values for key. */ + @Override + public V get(K key) { + Set ordinals = keyToOrdinal.get(key); + if (ordinals == null) { + return null; + } + return ordinals.stream().map(ordinalToValue::get).collect(Collectors.toList()).get(0); + } + + /** get set of values for key. */ + @Override + public Collection getAll(K key) { + Set ordinals = keyToOrdinal.get(key); + if (ordinals == null) { + return null; + } + return ordinals.stream().map(ordinalToValue::get).collect(Collectors.toList()); + } + + /** + * Inserts the tuple (key, value) into the multimap with automatic ordinal assignment. + * + *

A new ordinal is assigned if key/value pair does not exists. + * + *

If overwrite is true the existing key will be overwritten with value else value will be + * appended to the multimap. + */ + @Override + public boolean put(K key, V value, boolean overwrite) { + if (overwrite) { + removeAll(key); + } + Set ordinalSet = getOrdinals(key); + int nextOrdinal = ordinalToValue.size(); + ordinalToValue.put(nextOrdinal, value); + boolean changed = ordinalSet.add(nextOrdinal); + keyToOrdinal.put(key, ordinalSet); + return changed; + } + + @Override + public Collection values() { + return ordinalToValue.values(); + } + + @Override + public boolean containsKey(K key) { + return keyToOrdinal.containsKey(key); + } + + /** + * Removes the element corresponding to the key/value if exists with ordinal re-cycling. + * + *

The ordinal corresponding to the given key may be re-assigned to another tuple. It is + * important that consumer checks the ordinal value via {@link + * MultiMapWithOrdinal#getOrdinal(Object)} before attempting to look-up by ordinal. + * + *

If the multimap is changed return true. + */ + @Override + public synchronized boolean remove(K key, V value) { + Set removalSet = getOrdinals(key); + if (removalSet.isEmpty()) { + return false; + } + Optional removeValue = + removalSet.stream().map(ordinalToValue::get).filter(value::equals).findFirst(); + if (!removeValue.isPresent()) { + return false; + } + int removalOrdinal = removeKv(removalSet, key, value); + int lastOrdinal = ordinalToValue.size(); + if (lastOrdinal != removalOrdinal) { // we didn't remove the last ordinal + swapOrdinal(lastOrdinal, removalOrdinal); + } + return true; + } + + private void swapOrdinal(int lastOrdinal, int removalOrdinal) { + V swapOrdinalValue = ordinalToValue.remove(lastOrdinal); + ordinalToValue.put(removalOrdinal, swapOrdinalValue); + K swapOrdinalKey = + keyToOrdinal.entrySet().stream() + .filter(kv -> kv.getValue().stream().anyMatch(o -> o == lastOrdinal)) + .map(Map.Entry::getKey) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + "MultimapWithOrdinal in bad state")); + ordinalToValue.put(removalOrdinal, swapOrdinalValue); + Set swapSet = getOrdinals(swapOrdinalKey); + swapSet.remove(lastOrdinal); + swapSet.add(removalOrdinal); + keyToOrdinal.put(swapOrdinalKey, swapSet); + } + + private int removeKv(Set removalSet, K key, V value) { + Integer removalOrdinal = + removalSet.stream() + .filter(i -> ordinalToValue.get(i).equals(value)) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + "MultimapWithOrdinal in bad state")); + ordinalToValue.remove(removalOrdinal); + removalSet.remove(removalOrdinal); + if (removalSet.isEmpty()) { + keyToOrdinal.remove(key); + } else { + keyToOrdinal.put(key, removalSet); + } + return removalOrdinal; + } + + /** remove all entries of key. */ + @Override + public synchronized boolean removeAll(K key) { + Collection values = this.getAll(key); + if (values == null) { + return false; + } + for (V v : values) { + this.remove(key, v); + } + return true; + } + + @Override + public void clear() { + ordinalToValue.clear(); + keyToOrdinal.clear(); + } + + @Override + public Set keys() { + return keyToOrdinal.keySet(); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java index 06273fd098..956ae045d5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/ArrowUtils.java @@ -31,6 +31,7 @@ import org.apache.fluss.row.arrow.vectors.ArrowDoubleColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowFloatColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowIntColumnVector; +import org.apache.fluss.row.arrow.vectors.ArrowRowColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowSmallIntColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowTimeColumnVector; import org.apache.fluss.row.arrow.vectors.ArrowTimestampLtzColumnVector; @@ -48,6 +49,7 @@ import org.apache.fluss.row.arrow.writers.ArrowFieldWriter; import org.apache.fluss.row.arrow.writers.ArrowFloatWriter; import org.apache.fluss.row.arrow.writers.ArrowIntWriter; +import org.apache.fluss.row.arrow.writers.ArrowRowWriter; import org.apache.fluss.row.arrow.writers.ArrowSmallIntWriter; import org.apache.fluss.row.arrow.writers.ArrowTimeWriter; import org.apache.fluss.row.arrow.writers.ArrowTimestampLtzWriter; @@ -56,6 +58,7 @@ import org.apache.fluss.row.arrow.writers.ArrowVarBinaryWriter; import org.apache.fluss.row.arrow.writers.ArrowVarCharWriter; import org.apache.fluss.row.columnar.ColumnVector; +import org.apache.fluss.row.columnar.VectorizedColumnBatch; import org.apache.fluss.shaded.arrow.com.google.flatbuffers.FlatBufferBuilder; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.MessageHeader; import org.apache.fluss.shaded.arrow.org.apache.arrow.flatbuf.RecordBatch; @@ -84,6 +87,7 @@ import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorLoader; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.VectorSchemaRoot; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.ListVector; +import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.complex.StructVector; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.compression.NoCompressionCodec; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.ReadChannel; import org.apache.fluss.shaded.arrow.org.apache.arrow.vector.ipc.WriteChannel; @@ -131,7 +135,6 @@ import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -336,11 +339,20 @@ public static ArrowFieldWriter createArrowFieldWriter(FieldVector vector, DataTy FieldVector elementFieldVector = ((ListVector) vector).getDataVector(); return new ArrowArrayWriter( vector, ArrowUtils.createArrowFieldWriter(elementFieldVector, elementType)); + } else if (vector instanceof StructVector && dataType instanceof RowType) { + RowType rowType = (RowType) dataType; + StructVector structVector = (StructVector) vector; + List fieldVectors = structVector.getChildrenFromFields(); + ArrowFieldWriter[] fieldWriters = new ArrowFieldWriter[fieldVectors.size()]; + for (int i = 0; i < fieldVectors.size(); i++) { + fieldWriters[i] = + ArrowUtils.createArrowFieldWriter( + fieldVectors.get(i), rowType.getTypeAt(i)); + } + return new ArrowRowWriter(vector, fieldWriters); } else { throw new UnsupportedOperationException( - String.format( - "Unsupported type %s. Map and Row types will be supported in Issue #1973 and #1974.", - dataType)); + String.format("Unsupported type %s.", dataType)); } } @@ -387,11 +399,21 @@ private static ColumnVector createArrowColumnVector(ValueVector vector, DataType return new ArrowArrayColumnVector( listVector, ArrowUtils.createArrowColumnVector(listVector.getDataVector(), elementType)); + + } else if (vector instanceof StructVector && dataType instanceof RowType) { + RowType rowType = (RowType) dataType; + StructVector structVector = (StructVector) vector; + List fieldVectors = structVector.getChildrenFromFields(); + ColumnVector[] columnVectors = new ColumnVector[fieldVectors.size()]; + for (int i = 0; i < fieldVectors.size(); i++) { + columnVectors[i] = + ArrowUtils.createArrowColumnVector( + fieldVectors.get(i), rowType.getTypeAt(i)); + } + return new ArrowRowColumnVector(structVector, new VectorizedColumnBatch(columnVectors)); } else { throw new UnsupportedOperationException( - String.format( - "Unsupported type %s. Map and Row types will be supported in Issue #1973 and #1974.", - dataType)); + String.format("Unsupported type %s.", dataType)); } } @@ -412,19 +434,9 @@ private static Field toArrowField(String fieldName, DataType logicalType) { for (DataField field : rowType.getFields()) { children.add(toArrowField(field.getName(), field.getType())); } - } else if (logicalType instanceof MapType) { - MapType mapType = (MapType) logicalType; - Preconditions.checkArgument( - !mapType.getKeyType().isNullable(), "Map key type should be non-nullable"); - children = - Collections.singletonList( - new Field( - "items", - new FieldType(false, ArrowType.Struct.INSTANCE, null), - Arrays.asList( - toArrowField("key", mapType.getKeyType()), - toArrowField("value", mapType.getValueType())))); } + // TODO: Add Map type support in future + // else if (logicalType instanceof MapType) { ... } return new Field(fieldName, fieldType, children); } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayTest.java b/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayTest.java index 5f445c92af..d3054ead65 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayTest.java @@ -17,6 +17,7 @@ package org.apache.fluss.row; +import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.Test; @@ -103,7 +104,7 @@ public void testFromPrimitiveDoubleArray() { @Test public void testWriteAndReadInt() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 10); @@ -119,7 +120,7 @@ public void testWriteAndReadInt() { @Test public void testWriteAndReadString() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.writeString(0, BinaryString.fromString("hello")); @@ -133,7 +134,7 @@ public void testWriteAndReadString() { @Test public void testSetNull() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 8); writer.writeLong(0, 100L); @@ -151,7 +152,7 @@ public void testSetNull() { @Test public void testSetAndGetDecimal() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); Decimal decimal1 = Decimal.fromUnscaledLong(123, 5, 2); @@ -168,7 +169,7 @@ public void testSetAndGetDecimal() { @Test public void testSetAndGetTimestampNtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampNtz ts1 = TimestampNtz.fromMillis(1000L); @@ -185,7 +186,7 @@ public void testSetAndGetTimestampNtz() { @Test public void testSetAndGetTimestampLtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampLtz ts1 = TimestampLtz.fromEpochMillis(1000L); @@ -202,7 +203,7 @@ public void testSetAndGetTimestampLtz() { @Test public void testSetAndGetBinary() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); byte[] binary1 = {1, 2, 3}; @@ -217,32 +218,6 @@ public void testSetAndGetBinary() { assertThat(array.getBinary(1, 3)).isEqualTo(binary2); } - @Test - public void testCopy() { - int[] intArray = {1, 2, 3, 4, 5}; - BinaryArray original = BinaryArray.fromPrimitiveArray(intArray); - - BinaryArray copied = original.copy(); - - assertThat(copied.size()).isEqualTo(original.size()); - assertThat(copied.getInt(0)).isEqualTo(1); - assertThat(copied.getInt(4)).isEqualTo(5); - } - - @Test - public void testCopyWithReuse() { - int[] intArray = {1, 2, 3}; - BinaryArray original = BinaryArray.fromPrimitiveArray(intArray); - - BinaryArray reuse = new BinaryArray(); - BinaryArray copied = original.copy(reuse); - - assertThat(copied).isSameAs(reuse); - assertThat(copied.size()).isEqualTo(3); - assertThat(copied.getInt(0)).isEqualTo(1); - assertThat(copied.getInt(2)).isEqualTo(3); - } - @Test public void testHashCode() { int[] intArray = {1, 2, 3}; @@ -285,7 +260,7 @@ public void testFromInternalArrayAlreadyBinary() { @Test public void testAnyNull() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 10); @@ -304,7 +279,7 @@ public void testAnyNullWhenNoNulls() { @Test public void testToArrayWithNullThrowsException() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 10); @@ -327,7 +302,7 @@ public void testToObjectArray() { @Test public void testToObjectArrayWithNull() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 10); @@ -341,7 +316,7 @@ public void testToObjectArrayWithNull() { @Test public void testGetChar() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.writeString(0, BinaryString.fromString("hello")); @@ -354,7 +329,7 @@ public void testGetChar() { @Test public void testSetBoolean() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 1); writer.writeBoolean(0, true); @@ -369,7 +344,7 @@ public void testSetBoolean() { @Test public void testSetByte() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 1); writer.writeByte(0, (byte) 1); @@ -384,7 +359,7 @@ public void testSetByte() { @Test public void testSetShort() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 2); writer.writeShort(0, (short) 10); @@ -399,7 +374,7 @@ public void testSetShort() { @Test public void testSetFloat() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeFloat(0, 1.5f); @@ -414,7 +389,7 @@ public void testSetFloat() { @Test public void testSetDouble() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 8); writer.writeDouble(0, 1.1); @@ -454,7 +429,7 @@ public void testPointTo() { int[] intArray = {1, 2, 3}; BinaryArray array1 = BinaryArray.fromPrimitiveArray(intArray); - BinaryArray array2 = new BinaryArray(); + BinaryArray array2 = new PrimitiveBinaryArray(); array2.pointTo(array1.getSegments(), array1.getOffset(), array1.getSizeInBytes()); assertThat(array2.size()).isEqualTo(3); @@ -465,7 +440,7 @@ public void testPointTo() { @Test public void testHighPrecisionTimestamp() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampNtz ts1 = TimestampNtz.fromMillis(1000L, 123456); @@ -481,7 +456,7 @@ public void testHighPrecisionTimestamp() { @Test public void testHighPrecisionTimestampLtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampLtz ts1 = TimestampLtz.fromEpochMillis(1000L, 123456); @@ -497,7 +472,7 @@ public void testHighPrecisionTimestampLtz() { @Test public void testLargeDecimal() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); Decimal decimal1 = Decimal.fromBigDecimal(new java.math.BigDecimal("123.456"), 20, 3); @@ -513,7 +488,7 @@ public void testLargeDecimal() { @Test public void testSetNotNullAt() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 8); writer.setNullLong(0); @@ -529,7 +504,7 @@ public void testSetNotNullAt() { @Test public void testSetNullAt() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 8); writer.writeLong(0, 100L); @@ -567,7 +542,7 @@ public void testSetInt() { @Test public void testSetDecimalCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); Decimal decimal1 = Decimal.fromUnscaledLong(123, 5, 2); @@ -586,7 +561,7 @@ public void testSetDecimalCompact() { @Test public void testSetTimestampNtzCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampNtz ts1 = TimestampNtz.fromMillis(1000L); @@ -605,7 +580,7 @@ public void testSetTimestampNtzCompact() { @Test public void testSetTimestampLtzCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampLtz ts1 = TimestampLtz.fromEpochMillis(1000L); @@ -624,7 +599,7 @@ public void testSetTimestampLtzCompact() { @Test public void testToObjectArrayWithNulls() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 100); @@ -683,7 +658,7 @@ public void testEquals() { @Test public void testSetNullLong() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.setNullLong(0); writer.writeLong(1, 100L); @@ -695,7 +670,7 @@ public void testSetNullLong() { @Test public void testSetNullInt() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); writer.setNullInt(0); writer.writeInt(1, 100); @@ -707,7 +682,7 @@ public void testSetNullInt() { @Test public void testSetNullFloat() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); writer.setNullFloat(0); writer.writeFloat(1, 3.14f); @@ -719,7 +694,7 @@ public void testSetNullFloat() { @Test public void testSetNullDouble() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.setNullDouble(0); writer.writeDouble(1, 3.14159); @@ -731,7 +706,7 @@ public void testSetNullDouble() { @Test public void testSetNullBoolean() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 1); writer.setNullBoolean(0); writer.writeBoolean(1, true); @@ -743,7 +718,7 @@ public void testSetNullBoolean() { @Test public void testSetNullByte() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 1); writer.setNullByte(0); writer.writeByte(1, (byte) 42); @@ -755,7 +730,7 @@ public void testSetNullByte() { @Test public void testSetNullShort() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 2); writer.setNullShort(0); writer.writeShort(1, (short) 123); @@ -767,7 +742,7 @@ public void testSetNullShort() { @Test public void testGetString() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); writer.writeString(0, BinaryString.fromString("test")); writer.complete(); @@ -777,7 +752,7 @@ public void testGetString() { @Test public void testGetBytes() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); byte[] bytes = {1, 2, 3, 4, 5}; writer.writeBinary(0, bytes, 10); @@ -807,18 +782,6 @@ public void testGetSizeInBytes() { assertThat(size).isGreaterThan(0); } - @Test - public void testCopyWithMultipleTypes() { - BinaryArray array = BinaryArray.fromPrimitiveArray(new int[] {1, 2, 3, 4, 5}); - - BinaryArray copied = array.copy(); - - assertThat(copied).isNotSameAs(array); - assertThat(copied.size()).isEqualTo(5); - assertThat(copied.getInt(0)).isEqualTo(1); - assertThat(copied.getInt(4)).isEqualTo(5); - } - @Test public void testHashCodeConsistency() { BinaryArray array1 = BinaryArray.fromPrimitiveArray(new int[] {1, 2, 3}); @@ -829,7 +792,7 @@ public void testHashCodeConsistency() { @Test public void testSetDecimalNonCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); Decimal largeDecimal = @@ -849,7 +812,7 @@ public void testSetDecimalNonCompact() { @Test public void testSetTimestampNtzNonCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampNtz ts1 = TimestampNtz.fromMillis(1000L, 123456); @@ -868,7 +831,7 @@ public void testSetTimestampNtzNonCompact() { @Test public void testSetTimestampLtzNonCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampLtz ts1 = TimestampLtz.fromEpochMillis(1000L, 123456); @@ -919,7 +882,7 @@ public void testSetFloatDoubleMethods() { @Test public void testGetBinaryWithLength() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); byte[] data = {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; writer.writeBytes(0, data); @@ -931,7 +894,7 @@ public void testGetBinaryWithLength() { @Test public void testGetDecimalNonCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); Decimal largeDecimal = @@ -945,7 +908,7 @@ public void testGetDecimalNonCompact() { @Test public void testGetTimestampNonCompact() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampNtz tsNtz = TimestampNtz.fromMillis(1000L, 123456); @@ -966,4 +929,36 @@ public void testCalculateFixLengthPartSizeForArray() { assertThat(BinaryArray.calculateFixLengthPartSize(DataTypes.CHAR(10))).isEqualTo(8); assertThat(BinaryArray.calculateFixLengthPartSize(DataTypes.BINARY(20))).isEqualTo(8); } + + @Test + public void testPrimitiveBinaryArrayGetRowThrowsException() { + PrimitiveBinaryArray array = new PrimitiveBinaryArray(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); + writer.writeInt(0, 10); + writer.writeInt(1, 20); + writer.writeInt(2, 30); + writer.complete(); + + assertThatThrownBy(() -> array.getRow(0, 2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can not get nested row from array of primitive type"); + } + + @Test + public void testPrimitiveBinaryArrayGetArray() { + PrimitiveBinaryArray innerArray1 = new PrimitiveBinaryArray(); + BinaryArrayWriter innerWriter1 = new BinaryArrayWriter(innerArray1, 2, 4); + innerWriter1.writeInt(0, 10); + innerWriter1.writeInt(1, 20); + innerWriter1.complete(); + + PrimitiveBinaryArray outerArray = new PrimitiveBinaryArray(); + BinaryArrayWriter outerWriter = new BinaryArrayWriter(outerArray, 1, 8); + outerWriter.setOffsetAndSize(0, 0, innerArray1.getSizeInBytes()); + outerWriter.complete(); + + InternalArray result = outerArray.getArray(0); + assertThat(result).isNotNull(); + assertThat(result).isInstanceOf(PrimitiveBinaryArray.class); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayWriterTest.java b/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayWriterTest.java index 466346f987..a77367a479 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayWriterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/BinaryArrayWriterTest.java @@ -17,6 +17,7 @@ package org.apache.fluss.row; +import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.Test; @@ -28,7 +29,7 @@ public class BinaryArrayWriterTest { @Test public void testWriteAndReadAllTypes() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 10, 8); writer.writeBoolean(0, true); @@ -57,7 +58,7 @@ public void testWriteAndReadAllTypes() { @Test public void testReset() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 10); @@ -80,7 +81,7 @@ public void testReset() { @Test public void testSetNullMethods() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 8, 8); writer.setNullBoolean(0); @@ -108,7 +109,7 @@ public void testSetNullMethods() { @Test public void testSetOffsetAndSize() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.setOffsetAndSize(0, 100, 200); @@ -126,7 +127,7 @@ public void testSetOffsetAndSize() { @Test public void testGetFieldOffset() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 5, 8); int fieldOffset0 = writer.getFieldOffset(0); @@ -139,7 +140,7 @@ public void testGetFieldOffset() { @Test public void testGetNumElements() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 5, 4); assertThat(writer.getNumElements()).isEqualTo(5); @@ -147,7 +148,7 @@ public void testGetNumElements() { @Test public void testWriteChar() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.writeChar(0, BinaryString.fromString("hello"), 5); @@ -160,7 +161,7 @@ public void testWriteChar() { @Test public void testWriteBinary() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); byte[] binary1 = {1, 2, 3, 4}; @@ -176,7 +177,7 @@ public void testWriteBinary() { @Test public void testWriteTimestampNtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampNtz ts1 = TimestampNtz.fromMillis(1000L, 123456); @@ -192,7 +193,7 @@ public void testWriteTimestampNtz() { @Test public void testWriteTimestampLtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); TimestampLtz ts1 = TimestampLtz.fromEpochMillis(1000L, 123456); @@ -208,7 +209,7 @@ public void testWriteTimestampLtz() { @Test public void testWriteNaNFloat() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeFloat(0, Float.NaN); @@ -223,7 +224,7 @@ public void testWriteNaNFloat() { @Test public void testWriteNaNDouble() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 8); writer.writeDouble(0, Double.NaN); @@ -238,7 +239,7 @@ public void testWriteNaNDouble() { @Test public void testAfterGrow() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); writer.writeLong(0, 100L); @@ -251,7 +252,7 @@ public void testAfterGrow() { @Test public void testCreateNullSetter() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 10, 8); BinaryArrayWriter.NullSetter booleanSetter = @@ -314,7 +315,7 @@ public void testRoundNumberOfBytesToNearestWord() { @Test public void testWriteLargeString() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); String largeString = new String(new char[100]).replace('\0', 'a'); @@ -326,7 +327,7 @@ public void testWriteLargeString() { @Test public void testWriteSmallString() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); String smallString = "ab"; @@ -338,7 +339,7 @@ public void testWriteSmallString() { @Test public void testWriteLargeBinary() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); byte[] largeBinary = new byte[100]; @@ -354,7 +355,7 @@ public void testWriteLargeBinary() { @Test public void testSetNullBit() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 5, 4); writer.setNullBit(0); @@ -371,7 +372,7 @@ public void testSetNullBit() { @Test public void testMultipleResetCycles() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); for (int cycle = 0; cycle < 5; cycle++) { @@ -390,7 +391,7 @@ public void testMultipleResetCycles() { @Test public void testWriteNullDecimal() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); writer.writeDecimal(0, null, 20); @@ -401,7 +402,7 @@ public void testWriteNullDecimal() { @Test public void testWriteNullTimestampNtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); writer.writeTimestampNtz(0, null, 9); @@ -412,7 +413,7 @@ public void testWriteNullTimestampNtz() { @Test public void testWriteNullTimestampLtz() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 1, 8); writer.writeTimestampLtz(0, null, 9); diff --git a/fluss-common/src/test/java/org/apache/fluss/row/BinaryWriterTest.java b/fluss-common/src/test/java/org/apache/fluss/row/BinaryWriterTest.java index f2efb38814..d356eaaa85 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/BinaryWriterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/BinaryWriterTest.java @@ -17,6 +17,8 @@ package org.apache.fluss.row; +import org.apache.fluss.row.array.PrimitiveBinaryArray; +import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.Test; @@ -29,28 +31,24 @@ public class BinaryWriterTest { @Test public void testCreateValueSetterForAllTypes() { - BinaryWriter.ValueWriter booleanSetter = - BinaryWriter.createValueWriter(DataTypes.BOOLEAN()); - BinaryWriter.ValueWriter tinyintSetter = - BinaryWriter.createValueWriter(DataTypes.TINYINT()); - BinaryWriter.ValueWriter smallintSetter = - BinaryWriter.createValueWriter(DataTypes.SMALLINT()); - BinaryWriter.ValueWriter intSetter = BinaryWriter.createValueWriter(DataTypes.INT()); - BinaryWriter.ValueWriter bigintSetter = BinaryWriter.createValueWriter(DataTypes.BIGINT()); - BinaryWriter.ValueWriter floatSetter = BinaryWriter.createValueWriter(DataTypes.FLOAT()); - BinaryWriter.ValueWriter doubleSetter = BinaryWriter.createValueWriter(DataTypes.DOUBLE()); - BinaryWriter.ValueWriter stringSetter = BinaryWriter.createValueWriter(DataTypes.STRING()); - BinaryWriter.ValueWriter charSetter = BinaryWriter.createValueWriter(DataTypes.CHAR(10)); - BinaryWriter.ValueWriter binarySetter = - BinaryWriter.createValueWriter(DataTypes.BINARY(10)); + BinaryWriter.ValueWriter booleanSetter = createPrimitiveValueWriter(DataTypes.BOOLEAN()); + BinaryWriter.ValueWriter tinyintSetter = createPrimitiveValueWriter(DataTypes.TINYINT()); + BinaryWriter.ValueWriter smallintSetter = createPrimitiveValueWriter(DataTypes.SMALLINT()); + BinaryWriter.ValueWriter intSetter = createPrimitiveValueWriter(DataTypes.INT()); + BinaryWriter.ValueWriter bigintSetter = createPrimitiveValueWriter(DataTypes.BIGINT()); + BinaryWriter.ValueWriter floatSetter = createPrimitiveValueWriter(DataTypes.FLOAT()); + BinaryWriter.ValueWriter doubleSetter = createPrimitiveValueWriter(DataTypes.DOUBLE()); + BinaryWriter.ValueWriter stringSetter = createPrimitiveValueWriter(DataTypes.STRING()); + BinaryWriter.ValueWriter charSetter = createPrimitiveValueWriter(DataTypes.CHAR(10)); + BinaryWriter.ValueWriter binarySetter = createPrimitiveValueWriter(DataTypes.BINARY(10)); BinaryWriter.ValueWriter decimalSetter = - BinaryWriter.createValueWriter(DataTypes.DECIMAL(5, 2)); + createPrimitiveValueWriter(DataTypes.DECIMAL(5, 2)); BinaryWriter.ValueWriter timestampNtzSetter = - BinaryWriter.createValueWriter(DataTypes.TIMESTAMP(3)); + createPrimitiveValueWriter(DataTypes.TIMESTAMP(3)); BinaryWriter.ValueWriter timestampLtzSetter = - BinaryWriter.createValueWriter(DataTypes.TIMESTAMP_LTZ(3)); - BinaryWriter.ValueWriter dateSetter = BinaryWriter.createValueWriter(DataTypes.DATE()); - BinaryWriter.ValueWriter timeSetter = BinaryWriter.createValueWriter(DataTypes.TIME()); + createPrimitiveValueWriter(DataTypes.TIMESTAMP_LTZ(3)); + BinaryWriter.ValueWriter dateSetter = createPrimitiveValueWriter(DataTypes.DATE()); + BinaryWriter.ValueWriter timeSetter = createPrimitiveValueWriter(DataTypes.TIME()); assertThat(booleanSetter).isNotNull(); assertThat(tinyintSetter).isNotNull(); @@ -71,10 +69,10 @@ public void testCreateValueSetterForAllTypes() { @Test public void testValueSetterWithBooleanType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 1); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.BOOLEAN()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.BOOLEAN()); setter.writeValue(writer, 0, true); setter.writeValue(writer, 1, false); writer.complete(); @@ -85,10 +83,10 @@ public void testValueSetterWithBooleanType() { @Test public void testValueSetterWithIntType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.INT()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.INT()); setter.writeValue(writer, 0, 100); setter.writeValue(writer, 1, 200); writer.complete(); @@ -99,10 +97,10 @@ public void testValueSetterWithIntType() { @Test public void testValueSetterWithStringType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.STRING()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.STRING()); setter.writeValue(writer, 0, BinaryString.fromString("hello")); setter.writeValue(writer, 1, BinaryString.fromString("world")); writer.complete(); @@ -113,10 +111,10 @@ public void testValueSetterWithStringType() { @Test public void testValueSetterWithDecimalType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.DECIMAL(5, 2)); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.DECIMAL(5, 2)); setter.writeValue(writer, 0, Decimal.fromUnscaledLong(123, 5, 2)); setter.writeValue(writer, 1, Decimal.fromUnscaledLong(456, 5, 2)); writer.complete(); @@ -129,30 +127,18 @@ public void testValueSetterWithDecimalType() { public void testCreateValueSetterForMapThrowsException() { assertThatThrownBy( () -> - BinaryWriter.createValueWriter( + createPrimitiveValueWriter( DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()))) .isInstanceOf(UnsupportedOperationException.class) .hasMessageContaining("Map type is not supported yet"); } - @Test - public void testCreateValueSetterForRowThrowsException() { - assertThatThrownBy( - () -> - BinaryWriter.createValueWriter( - DataTypes.ROW( - DataTypes.FIELD("a", DataTypes.INT()), - DataTypes.FIELD("b", DataTypes.STRING())))) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Row type is not supported yet"); - } - @Test public void testValueSetterWithByteType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 1); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.TINYINT()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.TINYINT()); setter.writeValue(writer, 0, (byte) 10); setter.writeValue(writer, 1, (byte) 20); writer.complete(); @@ -163,10 +149,10 @@ public void testValueSetterWithByteType() { @Test public void testValueSetterWithShortType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 2); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.SMALLINT()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.SMALLINT()); setter.writeValue(writer, 0, (short) 100); setter.writeValue(writer, 1, (short) 200); writer.complete(); @@ -177,10 +163,10 @@ public void testValueSetterWithShortType() { @Test public void testValueSetterWithLongType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.BIGINT()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.BIGINT()); setter.writeValue(writer, 0, 1000L); setter.writeValue(writer, 1, 2000L); writer.complete(); @@ -191,10 +177,10 @@ public void testValueSetterWithLongType() { @Test public void testValueSetterWithFloatType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.FLOAT()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.FLOAT()); setter.writeValue(writer, 0, 1.5f); setter.writeValue(writer, 1, 2.5f); writer.complete(); @@ -205,10 +191,10 @@ public void testValueSetterWithFloatType() { @Test public void testValueSetterWithDoubleType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.DOUBLE()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.DOUBLE()); setter.writeValue(writer, 0, 1.1); setter.writeValue(writer, 1, 2.2); writer.complete(); @@ -219,10 +205,10 @@ public void testValueSetterWithDoubleType() { @Test public void testValueSetterWithCharType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.CHAR(5)); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.CHAR(5)); setter.writeValue(writer, 0, BinaryString.fromString("hello")); setter.writeValue(writer, 1, BinaryString.fromString("world")); writer.complete(); @@ -233,10 +219,10 @@ public void testValueSetterWithCharType() { @Test public void testValueSetterWithBinaryType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.BINARY(3)); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.BINARY(3)); setter.writeValue(writer, 0, new byte[] {1, 2, 3}); setter.writeValue(writer, 1, new byte[] {4, 5, 6}); writer.complete(); @@ -247,10 +233,10 @@ public void testValueSetterWithBinaryType() { @Test public void testValueSetterWithTimestampNtzType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.TIMESTAMP(3)); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.TIMESTAMP(3)); setter.writeValue(writer, 0, TimestampNtz.fromMillis(1000L)); setter.writeValue(writer, 1, TimestampNtz.fromMillis(2000L)); writer.complete(); @@ -261,11 +247,10 @@ public void testValueSetterWithTimestampNtzType() { @Test public void testValueSetterWithTimestampLtzType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 8); - BinaryWriter.ValueWriter setter = - BinaryWriter.createValueWriter(DataTypes.TIMESTAMP_LTZ(3)); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.TIMESTAMP_LTZ(3)); setter.writeValue(writer, 0, TimestampLtz.fromEpochMillis(1000L)); setter.writeValue(writer, 1, TimestampLtz.fromEpochMillis(2000L)); writer.complete(); @@ -276,10 +261,10 @@ public void testValueSetterWithTimestampLtzType() { @Test public void testValueSetterWithDateType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.DATE()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.DATE()); setter.writeValue(writer, 0, 18000); setter.writeValue(writer, 1, 18001); writer.complete(); @@ -290,10 +275,10 @@ public void testValueSetterWithDateType() { @Test public void testValueSetterWithTimeType() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 2, 4); - BinaryWriter.ValueWriter setter = BinaryWriter.createValueWriter(DataTypes.TIME()); + BinaryWriter.ValueWriter setter = createPrimitiveValueWriter(DataTypes.TIME()); setter.writeValue(writer, 0, 3600000); setter.writeValue(writer, 1, 7200000); writer.complete(); @@ -301,4 +286,9 @@ public void testValueSetterWithTimeType() { assertThat(array.getInt(0)).isEqualTo(3600000); assertThat(array.getInt(1)).isEqualTo(7200000); } + + private static BinaryWriter.ValueWriter createPrimitiveValueWriter(DataType elementType) { + // use null for row format if there is no nested row type + return BinaryWriter.createValueWriter(elementType, null); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/InternalArrayTest.java b/fluss-common/src/test/java/org/apache/fluss/row/InternalArrayTest.java index b98ab932b0..95e803b20f 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/InternalArrayTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/InternalArrayTest.java @@ -17,6 +17,7 @@ package org.apache.fluss.row; +import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.Test; @@ -233,7 +234,7 @@ public void testBinaryArrayWithElementGetter() { @Test public void testBinaryArrayWithNullElementGetter() { - BinaryArray array = new BinaryArray(); + BinaryArray array = new PrimitiveBinaryArray(); BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 4); writer.writeInt(0, 10); diff --git a/fluss-common/src/test/java/org/apache/fluss/row/TestInternalRowGenerator.java b/fluss-common/src/test/java/org/apache/fluss/row/TestInternalRowGenerator.java index 08448cc931..a9e08315dc 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/TestInternalRowGenerator.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/TestInternalRowGenerator.java @@ -32,6 +32,7 @@ import java.time.LocalTime; import java.util.Random; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.INDEXED; import static org.apache.fluss.row.BinaryString.fromString; /** Test all types and generate test internal row. */ @@ -66,16 +67,16 @@ public static RowType createAllRowType() { "f20", DataTypes.ARRAY(DataTypes.FLOAT().copy(false))), // vector embedding type new DataField( - "f21", DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))) // nested array - // TODO: Add Map and Row fields in Issue #1973 and #1974 - // new DataField("u", DataTypes.MAP(DataTypes.INT(), DataTypes.STRING())), - // new DataField( - // "v", - // DataTypes.ROW( - // new DataField("u1", DataTypes.INT()), - // new DataField("u2", DataTypes.ROW(DataTypes.INT())), - // new DataField("u3", DataTypes.STRING()))) - ); + "f21", + DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))), // nested array + // TODO: Add Map and Row fields in Issue #1973 + new DataField( + "f22", + DataTypes.ROW( + new DataField("u1", DataTypes.INT()), + new DataField( + "u2", DataTypes.ROW(new DataField("v1", DataTypes.INT()))), + new DataField("u3", DataTypes.STRING())))); } public static IndexedRow genIndexedRowForAllType() { @@ -84,7 +85,7 @@ public static IndexedRow genIndexedRowForAllType() { BinaryWriter.ValueWriter[] writers = new BinaryWriter.ValueWriter[dataTypes.length]; for (int i = 0; i < dataTypes.length; i++) { - writers[i] = BinaryWriter.createValueWriter(dataTypes[i]); + writers[i] = BinaryWriter.createValueWriter(dataTypes[i], INDEXED); } Random rnd = new Random(); @@ -147,10 +148,9 @@ public static IndexedRow genIndexedRowForAllType() { // GenericMap map = new GenericMap(javaMap); // setRandomNull(writers[20], writer, 20, rnd, map); - // TODO: Row type support will be added in Issue #1974 - // GenericRow innerRow = GenericRow.of(123); - // GenericRow genericRow = GenericRow.of(20, innerRow, BinaryString.fromString("Test")); - // setRandomNull(writers[21], writer, 21, rnd, genericRow); + GenericRow innerRow = GenericRow.of(22); + GenericRow genericRow = GenericRow.of(123, innerRow, BinaryString.fromString("Test")); + setRandomNull(writers[22], writer, 22, rnd, genericRow); IndexedRow row = new IndexedRow(dataTypes); row.pointTo(writer.segment(), 0, writer.position()); diff --git a/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java b/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java index 6836bcb60d..cb79d559bd 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/aligned/AlignedRowTest.java @@ -21,6 +21,10 @@ import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.BinaryWriter; import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.GenericArray; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; import org.apache.fluss.types.DataType; @@ -38,6 +42,7 @@ import java.util.Random; import java.util.Set; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.ALIGNED; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link AlignedRow}. */ @@ -546,7 +551,7 @@ public void testValueWriter() { AlignedRowWriter writer = new AlignedRowWriter(row); BinaryWriter.ValueWriter[] fieldSetters = new BinaryWriter.ValueWriter[10]; for (int i = 0; i < fieldTypes.length; i++) { - fieldSetters[i] = BinaryWriter.createValueWriter(fieldTypes[i]); + fieldSetters[i] = BinaryWriter.createValueWriter(fieldTypes[i], ALIGNED); } // Test static write method for different data types @@ -743,4 +748,72 @@ public void testComplexDataMix() { assertThat(row.getTimestampNtz(10, 9).toString()).contains("2021-01-01T00:00:00.000123456"); assertThat(row.isNullAt(11)).isTrue(); } + + @Test + public void testAlignedArrayGetRow() { + DataType rowType = + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.INT()), + DataTypes.FIELD("f2", DataTypes.STRING())); + DataType arrayType = DataTypes.ARRAY(rowType); + + AlignedRow outerRow = new AlignedRow(1); + AlignedRowWriter outerWriter = new AlignedRowWriter(outerRow); + + GenericRow row1 = GenericRow.of(100, BinaryString.fromString("nested")); + GenericRow row2 = GenericRow.of(200, BinaryString.fromString("nested2")); + GenericArray arrayData = GenericArray.of(row1, row2); + + BinaryWriter.ValueWriter arrayWriter = BinaryWriter.createValueWriter(arrayType, ALIGNED); + arrayWriter.writeValue(outerWriter, 0, arrayData); + outerWriter.complete(); + + InternalArray array = outerRow.getArray(0); + assertThat(array).isNotNull(); + assertThat(array.size()).isEqualTo(2); + + InternalRow nestedRow1 = array.getRow(0, 2); + assertThat(nestedRow1.getInt(0)).isEqualTo(100); + assertThat(nestedRow1.getString(1)).isEqualTo(BinaryString.fromString("nested")); + + InternalRow nestedRow2 = array.getRow(1, 2); + assertThat(nestedRow2.getInt(0)).isEqualTo(200); + assertThat(nestedRow2.getString(1)).isEqualTo(BinaryString.fromString("nested2")); + } + + @Test + public void testAlignedArrayGetNestedArray() { + DataType innerArrayType = DataTypes.ARRAY(DataTypes.INT()); + DataType outerArrayType = DataTypes.ARRAY(innerArrayType); + + AlignedRow outerRow = new AlignedRow(1); + AlignedRowWriter outerWriter = new AlignedRowWriter(outerRow); + + GenericArray innerArray1 = GenericArray.of(10, 20, 30); + GenericArray innerArray2 = GenericArray.of(40, 50, 60); + GenericArray outerArrayData = GenericArray.of(innerArray1, innerArray2); + + BinaryWriter.ValueWriter arrayWriter = + BinaryWriter.createValueWriter(outerArrayType, ALIGNED); + arrayWriter.writeValue(outerWriter, 0, outerArrayData); + outerWriter.complete(); + + InternalArray outerArray = outerRow.getArray(0); + assertThat(outerArray).isNotNull(); + assertThat(outerArray.size()).isEqualTo(2); + + InternalArray nestedArray1 = outerArray.getArray(0); + assertThat(nestedArray1).isNotNull(); + assertThat(nestedArray1.size()).isEqualTo(3); + assertThat(nestedArray1.getInt(0)).isEqualTo(10); + assertThat(nestedArray1.getInt(1)).isEqualTo(20); + assertThat(nestedArray1.getInt(2)).isEqualTo(30); + + InternalArray nestedArray2 = outerArray.getArray(1); + assertThat(nestedArray2).isNotNull(); + assertThat(nestedArray2.size()).isEqualTo(3); + assertThat(nestedArray2.getInt(0)).isEqualTo(40); + assertThat(nestedArray2.getInt(1)).isEqualTo(50); + assertThat(nestedArray2.getInt(2)).isEqualTo(60); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java b/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java index 16eb4a775a..fa8d14b3d2 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/arrow/ArrowReaderWriterTest.java @@ -89,11 +89,12 @@ class ArrowReaderWriterTest { DataTypes.TIMESTAMP_LTZ(9), DataTypes.ARRAY(DataTypes.INT()), DataTypes.ARRAY(DataTypes.FLOAT().copy(false)), // vector embedding type - DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING()))) // nested array - // TODO: Add Map and Row types in Issue #1973 and #1974 - // DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()), - // DataTypes.ROW(...) - ; + DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.STRING())), // nested array + // TODO: Add Map and Row types in Issue #1973 + DataTypes.ROW( + DataTypes.FIELD("i", DataTypes.INT()), + DataTypes.FIELD("r", NESTED_DATA_TYPE), + DataTypes.FIELD("s", DataTypes.STRING()))); private static final List TEST_DATA = Arrays.asList( @@ -124,10 +125,11 @@ class ArrowReaderWriterTest { GenericArray.of(0.1f, 1.1f, 2.2f, 3.3f, 4.4f, -0.5f, 6.6f), GenericArray.of( GenericArray.of(fromString("a"), fromString("b")), - GenericArray.of(fromString("c"), fromString("d")))), - // TODO: Add Map and Row test data in Issue #1973 and #1974 - // GenericMap.of(...), - // GenericRow.of(...)), + GenericArray.of(fromString("c"), fromString("d"))), + GenericRow.of( + 12, + GenericRow.of(34, fromString("56"), 78L), + fromString("910"))), GenericRow.of( false, (byte) 1, @@ -163,10 +165,11 @@ class ArrowReaderWriterTest { GenericArray.of( GenericArray.of(fromString("a"), null, fromString("c")), null, - GenericArray.of(fromString("hello"), fromString("world"))))); - // TODO: Add Map and Row test data in Issue #1973 and #1974 - // GenericMap.of(...), - // GenericRow.of(...))); + GenericArray.of(fromString("hello"), fromString("world"))), + GenericRow.of( + 12, + GenericRow.of(34, fromString("56"), 78L), + fromString("910")))); @Test void testReaderWriter() throws IOException { diff --git a/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowDeserializerTest.java b/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowDeserializerTest.java index c280c8ac8d..20bfb633d7 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowDeserializerTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowDeserializerTest.java @@ -25,6 +25,7 @@ import org.apache.fluss.row.InternalArray; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.array.PrimitiveBinaryArray; import org.apache.fluss.row.serializer.ArraySerializer; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypes; @@ -34,6 +35,7 @@ import java.math.BigDecimal; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.COMPACTED; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link CompactedRowDeserializer}. */ @@ -923,7 +925,7 @@ public void testDeserializeArrayType() { CompactedRowDeserializer deserializer = new CompactedRowDeserializer(types); BinaryArray intArray = BinaryArray.fromPrimitiveArray(new int[] {1, 2, 3, 4, 5}); - ArraySerializer arraySerializer = new ArraySerializer(DataTypes.INT()); + ArraySerializer arraySerializer = new ArraySerializer(DataTypes.INT(), COMPACTED); CompactedRowWriter writer = new CompactedRowWriter(types.length); writer.writeInt(100); @@ -948,14 +950,14 @@ public void testDeserializeArrayOfStrings() { DataType[] types = {DataTypes.ARRAY(DataTypes.STRING())}; CompactedRowDeserializer deserializer = new CompactedRowDeserializer(types); - BinaryArray strArray = new BinaryArray(); + BinaryArray strArray = new PrimitiveBinaryArray(); BinaryArrayWriter strArrayWriter = new BinaryArrayWriter(strArray, 3, 8); strArrayWriter.writeString(0, BinaryString.fromString("hello")); strArrayWriter.writeString(1, BinaryString.fromString("world")); strArrayWriter.writeString(2, BinaryString.fromString("test")); strArrayWriter.complete(); - ArraySerializer arraySerializer = new ArraySerializer(DataTypes.STRING()); + ArraySerializer arraySerializer = new ArraySerializer(DataTypes.STRING(), COMPACTED); CompactedRowWriter writer = new CompactedRowWriter(types.length); writer.writeArray(strArray, arraySerializer); diff --git a/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowTest.java b/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowTest.java index 72382f67ac..6c9b6a38b4 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowTest.java @@ -18,16 +18,22 @@ package org.apache.fluss.row.compacted; import org.apache.fluss.memory.MemorySegment; +import org.apache.fluss.row.BinaryRow.BinaryRowFormat; import org.apache.fluss.row.BinaryString; import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.GenericArray; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalArray; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; +import org.apache.fluss.row.serializer.ArraySerializer; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link CompactedRow}. */ public class CompactedRowTest { @@ -412,4 +418,88 @@ public void testLargeRow() { assertThat(row.getInt(i)).isEqualTo(i * 10); } } + + @Test + public void testCompactedArrayGetRowWithInvalidType() { + DataType arrayType = DataTypes.ARRAY(DataTypes.INT()); + DataType[] fieldTypes = {arrayType}; + + CompactedRow outerRow = new CompactedRow(fieldTypes); + CompactedRowWriter outerWriter = new CompactedRowWriter(fieldTypes.length); + + GenericArray arrayData = GenericArray.of(1, 2, 3); + ArraySerializer serializer = + new ArraySerializer(DataTypes.INT(), BinaryRowFormat.COMPACTED); + outerWriter.writeArray(arrayData, serializer); + + outerRow.pointTo(outerWriter.segment(), 0, outerWriter.position()); + + InternalArray array = outerRow.getArray(0); + assertThatThrownBy(() -> array.getRow(0, 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can not get row from Array of type"); + } + + @Test + public void testCompactedArrayGetRowWithWrongNumFields() { + DataType rowType = + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.INT()), + DataTypes.FIELD("f2", DataTypes.STRING())); + DataType arrayType = DataTypes.ARRAY(rowType); + DataType[] fieldTypes = {arrayType}; + + CompactedRow outerRow = new CompactedRow(fieldTypes); + CompactedRowWriter outerWriter = new CompactedRowWriter(fieldTypes.length); + + GenericRow innerRow = GenericRow.of(100, BinaryString.fromString("test")); + GenericArray arrayData = GenericArray.of(innerRow); + + ArraySerializer serializer = new ArraySerializer(rowType, BinaryRowFormat.COMPACTED); + outerWriter.writeArray(arrayData, serializer); + + outerRow.pointTo(outerWriter.segment(), 0, outerWriter.position()); + + InternalArray array = outerRow.getArray(0); + assertThatThrownBy(() -> array.getRow(0, 5)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unexpected number of fields"); + } + + @Test + public void testCompactedArrayGetNestedArray() { + DataType innerArrayType = DataTypes.ARRAY(DataTypes.INT()); + DataType outerArrayType = DataTypes.ARRAY(innerArrayType); + DataType[] fieldTypes = {outerArrayType}; + + CompactedRow outerRow = new CompactedRow(fieldTypes); + CompactedRowWriter outerWriter = new CompactedRowWriter(fieldTypes.length); + + GenericArray innerArray1 = GenericArray.of(10, 20, 30); + GenericArray innerArray2 = GenericArray.of(40, 50, 60); + GenericArray outerArrayData = GenericArray.of(innerArray1, innerArray2); + + ArraySerializer serializer = new ArraySerializer(innerArrayType, BinaryRowFormat.COMPACTED); + outerWriter.writeArray(outerArrayData, serializer); + + outerRow.pointTo(outerWriter.segment(), 0, outerWriter.position()); + + InternalArray outerArray = outerRow.getArray(0); + assertThat(outerArray).isNotNull(); + assertThat(outerArray.size()).isEqualTo(2); + + InternalArray nestedArray1 = outerArray.getArray(0); + assertThat(nestedArray1).isNotNull(); + assertThat(nestedArray1.size()).isEqualTo(3); + assertThat(nestedArray1.getInt(0)).isEqualTo(10); + assertThat(nestedArray1.getInt(1)).isEqualTo(20); + assertThat(nestedArray1.getInt(2)).isEqualTo(30); + + InternalArray nestedArray2 = outerArray.getArray(1); + assertThat(nestedArray2).isNotNull(); + assertThat(nestedArray2.size()).isEqualTo(3); + assertThat(nestedArray2.getInt(0)).isEqualTo(40); + assertThat(nestedArray2.getInt(1)).isEqualTo(50); + assertThat(nestedArray2.getInt(2)).isEqualTo(60); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowWriterTest.java b/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowWriterTest.java index 0dc9ea7bca..d84460ba4b 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowWriterTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/compacted/CompactedRowWriterTest.java @@ -33,6 +33,7 @@ import java.util.Random; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.COMPACTED; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link CompactedRowWriter}. */ @@ -97,7 +98,7 @@ public void testTypes() { new CompactedRowReader.FieldReader[allDataTypes.length]; for (int i = 0; i < allDataTypes.length; i++) { getters[i] = InternalRow.createFieldGetter(allDataTypes[i], i); - writers[i] = BinaryWriter.createValueWriter(allDataTypes[i]); + writers[i] = BinaryWriter.createValueWriter(allDataTypes[i], COMPACTED); readers[i] = CompactedRowReader.createFieldReader(allDataTypes[i]); } for (int i = 0; i < 1000; i++) { diff --git a/fluss-common/src/test/java/org/apache/fluss/row/encode/AlignedRowEncoderTest.java b/fluss-common/src/test/java/org/apache/fluss/row/encode/AlignedRowEncoderTest.java new file mode 100644 index 0000000000..be0c31f997 --- /dev/null +++ b/fluss-common/src/test/java/org/apache/fluss/row/encode/AlignedRowEncoderTest.java @@ -0,0 +1,252 @@ +/* + * 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.fluss.row.encode; + +import org.apache.fluss.row.BinaryRow; +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.types.DataType; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link AlignedRowEncoder}. */ +class AlignedRowEncoderTest { + + @Test + void testEncodeSimpleTypes() throws Exception { + DataType[] fieldTypes = + new DataType[] { + DataTypes.INT(), DataTypes.BIGINT(), DataTypes.STRING(), DataTypes.BOOLEAN() + }; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + encoder.encodeField(0, 100); + encoder.encodeField(1, 200L); + encoder.encodeField(2, BinaryString.fromString("test")); + encoder.encodeField(3, true); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getInt(0)).isEqualTo(100); + assertThat(row.getLong(1)).isEqualTo(200L); + assertThat(row.getString(2)).isEqualTo(BinaryString.fromString("test")); + assertThat(row.getBoolean(3)).isTrue(); + } + } + + @Test + void testEncodeMultipleRows() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.INT(), DataTypes.STRING()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + encoder.encodeField(0, 1); + encoder.encodeField(1, BinaryString.fromString("first")); + BinaryRow row1 = encoder.finishRow(); + BinaryRow row1Copy = row1.copy(); + + encoder.startNewRow(); + encoder.encodeField(0, 2); + encoder.encodeField(1, BinaryString.fromString("second")); + BinaryRow row2 = encoder.finishRow(); + + assertThat(row1Copy.getInt(0)).isEqualTo(1); + assertThat(row1Copy.getString(1)).isEqualTo(BinaryString.fromString("first")); + + assertThat(row2.getInt(0)).isEqualTo(2); + assertThat(row2.getString(1)).isEqualTo(BinaryString.fromString("second")); + } + } + + @Test + void testEncodeAllPrimitiveDataTypes() throws Exception { + DataType[] fieldTypes = + new DataType[] { + DataTypes.TINYINT(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.BOOLEAN(), + DataTypes.STRING(), + DataTypes.BYTES(), + DataTypes.DECIMAL(10, 2) + }; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + encoder.encodeField(0, (byte) 1); + encoder.encodeField(1, (short) 100); + encoder.encodeField(2, 1000); + encoder.encodeField(3, 10000L); + encoder.encodeField(4, 1.5f); + encoder.encodeField(5, 2.5); + encoder.encodeField(6, false); + encoder.encodeField(7, BinaryString.fromString("hello")); + encoder.encodeField(8, new byte[] {1, 2, 3}); + encoder.encodeField(9, Decimal.fromBigDecimal(new BigDecimal("123.45"), 10, 2)); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getByte(0)).isEqualTo((byte) 1); + assertThat(row.getShort(1)).isEqualTo((short) 100); + assertThat(row.getInt(2)).isEqualTo(1000); + assertThat(row.getLong(3)).isEqualTo(10000L); + assertThat(row.getFloat(4)).isEqualTo(1.5f); + assertThat(row.getDouble(5)).isEqualTo(2.5); + assertThat(row.getBoolean(6)).isFalse(); + assertThat(row.getString(7)).isEqualTo(BinaryString.fromString("hello")); + assertThat(row.getBytes(8)).isEqualTo(new byte[] {1, 2, 3}); + assertThat(row.getDecimal(9, 10, 2)) + .isEqualTo(Decimal.fromBigDecimal(new BigDecimal("123.45"), 10, 2)); + } + } + + @Test + void testEncodeWithNullValues() throws Exception { + DataType[] fieldTypes = + new DataType[] {DataTypes.INT(), DataTypes.STRING(), DataTypes.BIGINT()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + encoder.encodeField(0, 42); + encoder.encodeField(1, null); + encoder.encodeField(2, 100L); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getInt(0)).isEqualTo(42); + assertThat(row.isNullAt(1)).isTrue(); + assertThat(row.getLong(2)).isEqualTo(100L); + } + } + + @Test + void testEncodeTimestampTypes() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.TIMESTAMP_LTZ(3), DataTypes.INT()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + TimestampLtz timestampLtz = TimestampLtz.fromEpochMillis(1000000L); + + encoder.startNewRow(); + encoder.encodeField(0, timestampLtz); + encoder.encodeField(1, 999); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getTimestampLtz(0, 3)).isNotNull(); + assertThat(row.getInt(1)).isEqualTo(999); + } + } + + @Test + void testReuseEncoder() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.INT(), DataTypes.STRING()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + encoder.encodeField(0, 5); + encoder.encodeField(1, BinaryString.fromString("row5")); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getInt(0)).isEqualTo(5); + assertThat(row.getString(1)).isEqualTo(BinaryString.fromString("row5")); + + encoder.startNewRow(); + encoder.encodeField(0, 10); + encoder.encodeField(1, BinaryString.fromString("row10")); + row = encoder.finishRow(); + + assertThat(row.getInt(0)).isEqualTo(10); + assertThat(row.getString(1)).isEqualTo(BinaryString.fromString("row10")); + } + } + + @Test + void testEncodeEmptyRow() throws Exception { + DataType[] fieldTypes = new DataType[] {}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getFieldCount()).isEqualTo(0); + } + } + + @Test + void testEncodeSingleField() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.STRING()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + encoder.startNewRow(); + encoder.encodeField(0, BinaryString.fromString("single")); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getString(0)).isEqualTo(BinaryString.fromString("single")); + } + } + + @Test + void testClose() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.INT()}; + AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes); + encoder.close(); + } + + @Test + void testEncodeWithLargeStrings() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.STRING(), DataTypes.INT()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < 1000; i++) { + sb.append("test"); + } + String largeString = sb.toString(); + + encoder.startNewRow(); + encoder.encodeField(0, BinaryString.fromString(largeString)); + encoder.encodeField(1, 123); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getString(0)).isEqualTo(BinaryString.fromString(largeString)); + assertThat(row.getInt(1)).isEqualTo(123); + } + } + + @Test + void testEncodeWithBinaryData() throws Exception { + DataType[] fieldTypes = new DataType[] {DataTypes.BYTES(), DataTypes.BYTES()}; + try (AlignedRowEncoder encoder = new AlignedRowEncoder(fieldTypes)) { + byte[] data1 = new byte[100]; + byte[] data2 = new byte[200]; + for (int i = 0; i < data1.length; i++) { + data1[i] = (byte) i; + } + for (int i = 0; i < data2.length; i++) { + data2[i] = (byte) (i % 256); + } + + encoder.startNewRow(); + encoder.encodeField(0, data1); + encoder.encodeField(1, data2); + BinaryRow row = encoder.finishRow(); + + assertThat(row.getBytes(0)).isEqualTo(data1); + assertThat(row.getBytes(1)).isEqualTo(data2); + } + } +} diff --git a/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowReaderTest.java b/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowReaderTest.java index bc0a7ddc79..56a307563b 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowReaderTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowReaderTest.java @@ -21,6 +21,8 @@ import org.apache.fluss.row.BinaryWriter; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.GenericArray; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypes; import org.apache.fluss.utils.DateTimeUtils; @@ -33,11 +35,13 @@ import java.time.LocalDate; import java.time.LocalTime; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.INDEXED; import static org.apache.fluss.row.BinaryString.fromString; import static org.apache.fluss.row.TestInternalRowGenerator.createAllTypes; import static org.apache.fluss.row.indexed.IndexedRowTest.assertAllTypeEquals; import static org.apache.fluss.row.indexed.IndexedRowTest.genRecordForAllTypes; import static org.apache.fluss.testutils.InternalArrayAssert.assertThatArray; +import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; import static org.assertj.core.api.Assertions.assertThat; /** Test of {@link IndexedRowReader}. */ @@ -66,7 +70,7 @@ void testCreateFieldReader() { IndexedRowReader.FieldReader[] readers = new IndexedRowReader.FieldReader[dataTypes.length]; for (int i = 0; i < dataTypes.length; i++) { readers[i] = IndexedRowReader.createFieldReader(dataTypes[i]); - writers[i] = BinaryWriter.createValueWriter(dataTypes[i]); + writers[i] = BinaryWriter.createValueWriter(dataTypes[i], INDEXED); } IndexedRowWriter writer1 = new IndexedRowWriter(dataTypes); @@ -102,19 +106,32 @@ private void assertAllTypeEqualsForReader(IndexedRowReader reader) { assertThat(reader.readTimestampNtz(5).toString()).isEqualTo("2023-10-25T12:01:13.182"); assertThat(reader.readTimestampLtz(1).toString()).isEqualTo("2023-10-25T12:01:13.182Z"); assertThat(reader.readTimestampLtz(5).toString()).isEqualTo("2023-10-25T12:01:13.182Z"); - assertThatArray(reader.readArray()) + assertThatArray(reader.readArray(dataTypes[19])) .withElementType(DataTypes.INT()) .isEqualTo(GenericArray.of(1, 2, 3, 4, 5, -11, null, 444, 102234)); - assertThatArray(reader.readArray()) - .withElementType(DataTypes.FLOAT().copy(false)) + assertThatArray(reader.readArray(dataTypes[20])) + .withElementType(DataTypes.FLOAT()) .isEqualTo( GenericArray.of(0.1f, 1.1f, -0.5f, 6.6f, Float.MAX_VALUE, Float.MIN_VALUE)); - assertThatArray(reader.readArray()) + assertThatArray(reader.readArray(dataTypes[21])) .withElementType(DataTypes.ARRAY(DataTypes.STRING())) .isEqualTo( GenericArray.of( GenericArray.of(fromString("a"), null, fromString("c")), null, GenericArray.of(fromString("hello"), fromString("world")))); + InternalRow nestedRow = + reader.readRow(dataTypes[22].getChildren().toArray(new DataType[0])); + GenericRow expectedInnerRow = GenericRow.of(22); + GenericRow expectedNestedRow = GenericRow.of(123, expectedInnerRow, fromString("Test")); + assertThatRow(nestedRow) + .withSchema( + DataTypes.ROW( + DataTypes.FIELD("u1", DataTypes.INT()), + DataTypes.FIELD( + "u2", + DataTypes.ROW(DataTypes.FIELD("v1", DataTypes.INT()))), + DataTypes.FIELD("u3", DataTypes.STRING()))) + .isEqualTo(expectedNestedRow); } } diff --git a/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowTest.java b/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowTest.java index 390998475b..3e4066991a 100644 --- a/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/row/indexed/IndexedRowTest.java @@ -21,10 +21,11 @@ import org.apache.fluss.row.BinaryWriter; import org.apache.fluss.row.Decimal; import org.apache.fluss.row.GenericArray; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalArray; import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.TimestampLtz; import org.apache.fluss.row.TimestampNtz; -import org.apache.fluss.row.serializer.ArraySerializer; import org.apache.fluss.types.DataType; import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.IntType; @@ -39,9 +40,11 @@ import java.time.LocalDate; import java.time.LocalTime; +import static org.apache.fluss.row.BinaryRow.BinaryRowFormat.INDEXED; import static org.apache.fluss.row.BinaryString.fromString; import static org.apache.fluss.row.TestInternalRowGenerator.createAllTypes; import static org.apache.fluss.testutils.InternalArrayAssert.assertThatArray; +import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -72,9 +75,7 @@ void testWriterAndIndexedRowGetter() { assertAllTypeEquals(row); - assertThat(row.getHeaderSizeInBytes()).isEqualTo(27); - assertThat(row.getSizeInBytes()).isEqualTo(309); - assertThat(row.getFieldCount()).isEqualTo(22); + assertThat(row.getFieldCount()).isEqualTo(23); assertThat(row.anyNull()).isFalse(); assertThat(row.anyNull(new int[] {0, 1})).isFalse(); } @@ -125,7 +126,7 @@ void testCreateFieldWriter() { BinaryWriter.ValueWriter[] writers = new BinaryWriter.ValueWriter[dataTypes.length]; for (int i = 0; i < dataTypes.length; i++) { fieldGetter[i] = InternalRow.createFieldGetter(dataTypes[i], i); - writers[i] = BinaryWriter.createValueWriter(dataTypes[i]); + writers[i] = BinaryWriter.createValueWriter(dataTypes[i], INDEXED); } IndexedRowWriter writer1 = new IndexedRowWriter(dataTypes); @@ -134,7 +135,7 @@ void testCreateFieldWriter() { } IndexedRow row1 = new IndexedRow(dataTypes); - row1.pointTo(writer1.segment(), 0, writer.position()); + row1.pointTo(writer1.segment(), 0, writer1.position()); assertAllTypeEquals(row1); } @@ -176,37 +177,50 @@ void testProjectRow() { public static IndexedRowWriter genRecordForAllTypes(DataType[] dataTypes) { IndexedRowWriter writer = new IndexedRowWriter(dataTypes); - writer.writeBoolean(true); - writer.writeByte((byte) 2); - writer.writeShort(Short.parseShort("10")); - writer.writeInt(100); - writer.writeLong(new BigInteger("12345678901234567890").longValue()); - writer.writeFloat(Float.parseFloat("13.2")); - writer.writeDouble(Double.parseDouble("15.21")); - writer.writeInt((int) TypeUtils.castFromString("2023-10-25", DataTypes.DATE())); - writer.writeInt((int) TypeUtils.castFromString("09:30:00.0", DataTypes.TIME())); - writer.writeBinary("1234567890".getBytes(), 20); - writer.writeBytes("20".getBytes()); - writer.writeChar(fromString("1"), 2); - writer.writeString(fromString("hello")); - writer.writeDecimal(Decimal.fromUnscaledLong(9, 5, 2), 5); - writer.writeDecimal(Decimal.fromBigDecimal(new BigDecimal(10), 20, 0), 20); - writer.writeTimestampNtz(TimestampNtz.fromMillis(1698235273182L), 1); - writer.writeTimestampNtz(TimestampNtz.fromMillis(1698235273182L), 5); - writer.writeTimestampLtz(TimestampLtz.fromEpochMillis(1698235273182L), 1); - writer.writeTimestampLtz(TimestampLtz.fromEpochMillis(1698235273182L), 5); - writer.writeArray( - GenericArray.of(1, 2, 3, 4, 5, -11, null, 444, 102234), - new ArraySerializer(DataTypes.INT())); - writer.writeArray( - GenericArray.of(0.1f, 1.1f, -0.5f, 6.6f, Float.MAX_VALUE, Float.MIN_VALUE), - new ArraySerializer(DataTypes.FLOAT().copy(false))); - writer.writeArray( + + BinaryWriter.ValueWriter[] writers = new BinaryWriter.ValueWriter[dataTypes.length]; + for (int i = 0; i < dataTypes.length; i++) { + writers[i] = BinaryWriter.createValueWriter(dataTypes[i], INDEXED); + } + + writers[0].writeValue(writer, 0, true); + writers[1].writeValue(writer, 1, (byte) 2); + writers[2].writeValue(writer, 2, Short.parseShort("10")); + writers[3].writeValue(writer, 3, 100); + writers[4].writeValue(writer, 4, new BigInteger("12345678901234567890").longValue()); + writers[5].writeValue(writer, 5, Float.parseFloat("13.2")); + writers[6].writeValue(writer, 6, Double.parseDouble("15.21")); + writers[7].writeValue( + writer, 7, (int) TypeUtils.castFromString("2023-10-25", DataTypes.DATE())); + writers[8].writeValue( + writer, 8, (int) TypeUtils.castFromString("09:30:00.0", DataTypes.TIME())); + writers[9].writeValue(writer, 9, "1234567890".getBytes()); + writers[10].writeValue(writer, 10, "20".getBytes()); + writers[11].writeValue(writer, 11, fromString("1")); + writers[12].writeValue(writer, 12, fromString("hello")); + writers[13].writeValue(writer, 13, Decimal.fromUnscaledLong(9, 5, 2)); + writers[14].writeValue(writer, 14, Decimal.fromBigDecimal(new BigDecimal(10), 20, 0)); + writers[15].writeValue(writer, 15, TimestampNtz.fromMillis(1698235273182L)); + writers[16].writeValue(writer, 16, TimestampNtz.fromMillis(1698235273182L)); + writers[17].writeValue(writer, 17, TimestampLtz.fromEpochMillis(1698235273182L)); + writers[18].writeValue(writer, 18, TimestampLtz.fromEpochMillis(1698235273182L)); + writers[19].writeValue(writer, 19, GenericArray.of(1, 2, 3, 4, 5, -11, null, 444, 102234)); + writers[20].writeValue( + writer, + 20, + GenericArray.of(0.1f, 1.1f, -0.5f, 6.6f, Float.MAX_VALUE, Float.MIN_VALUE)); + writers[21].writeValue( + writer, + 21, GenericArray.of( GenericArray.of(fromString("a"), null, fromString("c")), null, - GenericArray.of(fromString("hello"), fromString("world"))), - new ArraySerializer(DataTypes.ARRAY(DataTypes.STRING()))); + GenericArray.of(fromString("hello"), fromString("world")))); + + GenericRow innerRow = GenericRow.of(22); + GenericRow nestedRow = GenericRow.of(123, innerRow, fromString("Test")); + writers[22].writeValue(writer, 22, nestedRow); + return writer; } @@ -245,5 +259,133 @@ public static void assertAllTypeEquals(InternalRow row) { GenericArray.of(fromString("a"), null, fromString("c")), null, GenericArray.of(fromString("hello"), fromString("world")))); + GenericRow expectedInnerRow = GenericRow.of(22); + GenericRow expectedNestedRow = GenericRow.of(123, expectedInnerRow, fromString("Test")); + assertThatRow(row.getRow(22, 3)) + .withSchema( + DataTypes.ROW( + DataTypes.FIELD("u1", DataTypes.INT()), + DataTypes.FIELD( + "u2", + DataTypes.ROW(DataTypes.FIELD("v1", DataTypes.INT()))), + DataTypes.FIELD("u3", DataTypes.STRING()))) + .isEqualTo(expectedNestedRow); + } + + @Test + void testIndexedArrayGetRow() { + DataType rowType = + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.INT()), + DataTypes.FIELD("f2", DataTypes.STRING())); + + DataType[] fieldTypes = new DataType[] {DataTypes.ARRAY(rowType)}; + IndexedRow outerRow = new IndexedRow(fieldTypes); + IndexedRowWriter outerWriter = new IndexedRowWriter(fieldTypes); + + GenericRow row1 = GenericRow.of(100, fromString("first")); + GenericRow row2 = GenericRow.of(200, fromString("second")); + GenericArray arrayData = GenericArray.of(row1, row2); + + BinaryWriter.ValueWriter arrayWriter = + BinaryWriter.createValueWriter(fieldTypes[0], INDEXED); + arrayWriter.writeValue(outerWriter, 0, arrayData); + + outerRow.pointTo(outerWriter.segment(), 0, outerWriter.position()); + + InternalArray array = outerRow.getArray(0); + assertThat(array).isNotNull(); + assertThat(array.size()).isEqualTo(2); + + InternalRow nestedRow1 = array.getRow(0, 2); + assertThat(nestedRow1.getInt(0)).isEqualTo(100); + assertThat(nestedRow1.getString(1)).isEqualTo(fromString("first")); + + InternalRow nestedRow2 = array.getRow(1, 2); + assertThat(nestedRow2.getInt(0)).isEqualTo(200); + assertThat(nestedRow2.getString(1)).isEqualTo(fromString("second")); + } + + @Test + void testIndexedArrayGetRowWithInvalidType() { + DataType[] fieldTypes = new DataType[] {DataTypes.ARRAY(DataTypes.INT())}; + IndexedRow outerRow = new IndexedRow(fieldTypes); + IndexedRowWriter outerWriter = new IndexedRowWriter(fieldTypes); + + GenericArray arrayData = GenericArray.of(1, 2, 3); + BinaryWriter.ValueWriter arrayWriter = + BinaryWriter.createValueWriter(fieldTypes[0], INDEXED); + arrayWriter.writeValue(outerWriter, 0, arrayData); + + outerRow.pointTo(outerWriter.segment(), 0, outerWriter.position()); + + InternalArray array = outerRow.getArray(0); + assertThatThrownBy(() -> array.getRow(0, 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can not get row from Array of type"); + } + + @Test + void testIndexedArrayGetRowWithWrongNumFields() { + DataType rowType = + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.INT()), + DataTypes.FIELD("f2", DataTypes.STRING())); + + DataType[] fieldTypes = new DataType[] {DataTypes.ARRAY(rowType)}; + IndexedRow outerRow = new IndexedRow(fieldTypes); + IndexedRowWriter outerWriter = new IndexedRowWriter(fieldTypes); + + GenericRow row1 = GenericRow.of(100, fromString("first")); + GenericArray arrayData = GenericArray.of(row1); + + BinaryWriter.ValueWriter arrayWriter = + BinaryWriter.createValueWriter(fieldTypes[0], INDEXED); + arrayWriter.writeValue(outerWriter, 0, arrayData); + + outerRow.pointTo(outerWriter.segment(), 0, outerWriter.position()); + + InternalArray array = outerRow.getArray(0); + assertThatThrownBy(() -> array.getRow(0, 5)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unexpected number of fields"); + } + + @Test + void testIndexedArrayGetNestedArray() { + DataType innerArrayType = DataTypes.ARRAY(DataTypes.INT()); + DataType outerArrayType = DataTypes.ARRAY(innerArrayType); + + DataType[] fieldTypes = new DataType[] {outerArrayType}; + IndexedRow row = new IndexedRow(fieldTypes); + IndexedRowWriter rowWriter = new IndexedRowWriter(fieldTypes); + + GenericArray innerArray1 = GenericArray.of(1, 2, 3); + GenericArray innerArray2 = GenericArray.of(4, 5, 6); + GenericArray outerArrayData = GenericArray.of(innerArray1, innerArray2); + + BinaryWriter.ValueWriter arrayWriter = + BinaryWriter.createValueWriter(fieldTypes[0], INDEXED); + arrayWriter.writeValue(rowWriter, 0, outerArrayData); + + row.pointTo(rowWriter.segment(), 0, rowWriter.position()); + + InternalArray outerArray = row.getArray(0); + assertThat(outerArray).isNotNull(); + assertThat(outerArray.size()).isEqualTo(2); + + InternalArray nestedArray1 = outerArray.getArray(0); + assertThat(nestedArray1).isNotNull(); + assertThat(nestedArray1.size()).isEqualTo(3); + assertThat(nestedArray1.getInt(0)).isEqualTo(1); + assertThat(nestedArray1.getInt(1)).isEqualTo(2); + assertThat(nestedArray1.getInt(2)).isEqualTo(3); + + InternalArray nestedArray2 = outerArray.getArray(1); + assertThat(nestedArray2).isNotNull(); + assertThat(nestedArray2.size()).isEqualTo(3); + assertThat(nestedArray2.getInt(0)).isEqualTo(4); + assertThat(nestedArray2.getInt(1)).isEqualTo(5); + assertThat(nestedArray2.getInt(2)).isEqualTo(6); } } diff --git a/fluss-common/src/test/java/org/apache/fluss/testutils/DataTestUtils.java b/fluss-common/src/test/java/org/apache/fluss/testutils/DataTestUtils.java index 3cb1834f15..662faca600 100644 --- a/fluss-common/src/test/java/org/apache/fluss/testutils/DataTestUtils.java +++ b/fluss-common/src/test/java/org/apache/fluss/testutils/DataTestUtils.java @@ -104,6 +104,16 @@ public static GenericRow row(Object... objects) { return row; } + public static GenericRow row(RowType rowType, Object... objects) { + GenericRow row = new GenericRow(objects.length); + List fieldTypes = rowType.getChildren(); + for (int i = 0; i < objects.length; i++) { + Object value = toInternalObject(objects[i], fieldTypes.get(i)); + row.setField(i, value); + } + return row; + } + private static Object toInternalObject(Object obj) { if (obj == null) { return null; @@ -124,6 +134,45 @@ private static Object toInternalObject(Object obj) { } } + private static Object toInternalObject(Object obj, DataType dataType) { + if (obj == null) { + return null; + } + + DataTypeRoot typeRoot = dataType.getTypeRoot(); + + if (typeRoot == DataTypeRoot.ROW) { + if (obj instanceof Object[]) { + return row((RowType) dataType, (Object[]) obj); + } + throw new IllegalArgumentException( + "Expected Object[] for ROW type, but got: " + obj.getClass().getSimpleName()); + } + + if (typeRoot == DataTypeRoot.ARRAY) { + if (obj instanceof int[]) { + return new GenericArray((int[]) obj); + } + if (obj instanceof Object[]) { + DataType elementType = dataType.getChildren().get(0); + Object[] array = (Object[]) obj; + Object[] internalArray = new Object[array.length]; + for (int j = 0; j < array.length; j++) { + internalArray[j] = toInternalObject(array[j], elementType); + } + return new GenericArray(internalArray); + } + throw new IllegalArgumentException( + "Expected array for ARRAY type, but got: " + obj.getClass().getSimpleName()); + } + + if (obj instanceof String) { + return BinaryString.fromString((String) obj); + } + + return obj; + } + public static CompactedRow compactedRow(RowType rowType, Object[] objects) { return genCompacted(rowType, objects); } @@ -474,7 +523,7 @@ public static MemoryLogRecords createMemoryLogRecords( throws Exception { if (logFormat == LogFormat.ARROW) { List rows = - objects.stream().map(DataTestUtils::row).collect(Collectors.toList()); + objects.stream().map(objs -> row(rowType, objs)).collect(Collectors.toList()); return createArrowMemoryLogRecords( rowType, offsetBase, diff --git a/fluss-common/src/test/java/org/apache/fluss/testutils/InternalRowAssert.java b/fluss-common/src/test/java/org/apache/fluss/testutils/InternalRowAssert.java index 3e6a672ca0..eade40e89d 100644 --- a/fluss-common/src/test/java/org/apache/fluss/testutils/InternalRowAssert.java +++ b/fluss-common/src/test/java/org/apache/fluss/testutils/InternalRowAssert.java @@ -28,6 +28,8 @@ import org.assertj.core.api.AbstractAssert; import static org.apache.fluss.types.DataTypeRoot.ARRAY; +import static org.apache.fluss.types.DataTypeRoot.MAP; +import static org.apache.fluss.types.DataTypeRoot.ROW; import static org.assertj.core.api.Assertions.assertThat; /** Extend assertj assertions to easily assert {@link InternalRow}. */ @@ -76,12 +78,20 @@ public InternalRowAssert isEqualTo(InternalRow expected) { Object actualField = fieldGetters[i].getFieldOrNull(actual); Object expectedField = fieldGetters[i].getFieldOrNull(expected); - // Special handling for Array types to compare content not instance + // Special handling for Array, Map, and Row types to compare content not instance if (fieldType.getTypeRoot() == ARRAY) { InternalArrayAssert.assertThatArray((InternalArray) actualField) .withElementType(((ArrayType) fieldType).getElementType()) .as("InternalRow#get" + fieldType.getTypeRoot() + "(" + i + ")") .isEqualTo((InternalArray) expectedField); + } else if (fieldType.getTypeRoot() == MAP) { + // TODO: Add Map type assertion support in future + throw new UnsupportedOperationException("Map type not supported yet"); + } else if (fieldType.getTypeRoot() == ROW) { + assertThatRow((InternalRow) actualField) + .withSchema((RowType) fieldType) + .as("InternalRow#get" + fieldType.getTypeRoot() + "(" + i + ")") + .isEqualTo((InternalRow) expectedField); } else { assertThat(actualField) .as("InternalRow#get" + fieldType.getTypeRoot() + "(" + i + ")") diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java index fc1188c709..c4f22a3346 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussArray.java @@ -165,6 +165,11 @@ public InternalArray getArray(int pos) { return new FlinkAsFlussArray(flinkArray.getArray(pos)); } + @Override + public InternalRow getRow(int pos, int numFields) { + return new FlinkAsFlussRow(flinkArray.getRow(pos, numFields)); + } + @SuppressWarnings("unchecked") public T[] toObjectArray(DataType elementType) { Class elementClass = (Class) InternalRow.getDataClass(elementType); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java index 003e910eaa..a5b0bc3ada 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/row/FlinkAsFlussRow.java @@ -35,6 +35,10 @@ public class FlinkAsFlussRow implements InternalRow { public FlinkAsFlussRow() {} + public FlinkAsFlussRow(RowData flinkRow) { + this.flinkRow = flinkRow; + } + public FlinkAsFlussRow replace(RowData flinkRow) { this.flinkRow = flinkRow; return this; @@ -139,6 +143,8 @@ public InternalArray getArray(int pos) { return new FlinkAsFlussArray(flinkRow.getArray(pos)); } - // TODO: Support Map type conversion from Flink to Fluss - // TODO: Support Row type conversion from Flink to Fluss + @Override + public InternalRow getRow(int pos, int numFields) { + return new FlinkAsFlussRow(flinkRow.getRow(pos, numFields)); + } } diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java index 2949c9b8d1..dd23e8ec39 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverter.java @@ -179,6 +179,12 @@ private FlussSerializationConverter createInternalConverter(LogicalType flinkDat }; case ARRAY: return flinkField -> new FlinkAsFlussArray((ArrayData) flinkField); + case MAP: + case MULTISET: + // TODO: Add Map type support in future + throw new UnsupportedOperationException("Map type not supported yet"); + case ROW: + return flinkField -> new FlinkAsFlussRow((RowData) flinkField); default: throw new UnsupportedOperationException( "Fluss Unsupported data type: " + flinkDataType); diff --git a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java index e52a83a820..d27e660405 100644 --- a/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java +++ b/fluss-flink/fluss-flink-common/src/main/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverter.java @@ -75,12 +75,12 @@ private RowData toFlinkRowData(InternalRow flussRow, RowKind rowKind) { /** * Create a nullable runtime {@link FlussDeserializationConverter} from given {@link DataType}. */ - protected FlussDeserializationConverter createNullableInternalConverter( + protected static FlussDeserializationConverter createNullableInternalConverter( DataType flussDataType) { return wrapIntoNullableInternalConverter(createInternalConverter(flussDataType)); } - protected FlussDeserializationConverter wrapIntoNullableInternalConverter( + protected static FlussDeserializationConverter wrapIntoNullableInternalConverter( FlussDeserializationConverter flussDeserializationConverter) { return val -> { if (val == null) { @@ -108,7 +108,7 @@ public interface FlussDeserializationConverter extends Serializable { } // TODO: use flink row type - private FlussDeserializationConverter createInternalConverter(DataType flussDataType) { + static FlussDeserializationConverter createInternalConverter(DataType flussDataType) { switch (flussDataType.getTypeRoot()) { case BOOLEAN: case TINYINT: @@ -161,6 +161,28 @@ private FlussDeserializationConverter createInternalConverter(DataType flussData } return new GenericArrayData(flinkArray); }; + case MAP: + // TODO: Add Map type support in future + throw new UnsupportedOperationException("Map type not supported yet"); + case ROW: + RowType rowType = (RowType) flussDataType; + int fieldCount = rowType.getFieldCount(); + InternalRow.FieldGetter[] fieldGetters = new InternalRow.FieldGetter[fieldCount]; + FlussDeserializationConverter[] fieldConverters = + new FlussDeserializationConverter[fieldCount]; + for (int i = 0; i < fieldCount; i++) { + fieldGetters[i] = InternalRow.createFieldGetter(rowType.getTypeAt(i), i); + fieldConverters[i] = createNullableInternalConverter(rowType.getTypeAt(i)); + } + return (flussField) -> { + InternalRow flussRow = (InternalRow) flussField; + GenericRowData flinkRowData = new GenericRowData(fieldCount); + for (int i = 0; i < fieldCount; i++) { + Object flussFieldValue = fieldGetters[i].getFieldOrNull(flussRow); + flinkRowData.setField(i, fieldConverters[i].deserialize(flussFieldValue)); + } + return flinkRowData; + }; default: throw new UnsupportedOperationException("Unsupported data type: " + flussDataType); } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkComplexTypeITCase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkComplexTypeITCase.java index efa408a7fd..7df2983874 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkComplexTypeITCase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/FlinkComplexTypeITCase.java @@ -278,6 +278,32 @@ void testArrayTypesInPrimaryKeyTable() throws Exception { assertResultsIgnoreOrder(collected, expected, true); } + @Test + void testRowTypesInLogTable() throws Exception { + tEnv.executeSql( + "create table row_log_test (" + + "id int, " + + "simple_row row, " + + "nested_row row, v string>, " + + "array_of_rows array>" + + ") with ('bucket.num' = '3')"); + + tEnv.executeSql( + "INSERT INTO row_log_test VALUES " + + "(1, ROW(10, 'hello'), ROW(20, ROW(30, 'nested'), 'row1'), " + + "ARRAY[ROW(1, 'a'), ROW(2, 'b')]), " + + "(2, ROW(40, 'world'), ROW(50, ROW(60, 'test'), 'row2'), " + + "ARRAY[ROW(3, 'c')])") + .await(); + + CloseableIterator rowIter = tEnv.executeSql("select * from row_log_test").collect(); + List expectedRows = + Arrays.asList( + "+I[1, +I[10, hello], +I[20, +I[30, nested], row1], [+I[1, a], +I[2, b]]]", + "+I[2, +I[40, world], +I[50, +I[60, test], row2], [+I[3, c]]]"); + assertResultsIgnoreOrder(rowIter, expectedRows, true); + } + @Test void testExceptionsForArrayTypeUsage() { assertThatThrownBy( diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverterTest.java index b81241d45e..c9379d8c1b 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkRowToFlussRowConverterTest.java @@ -52,7 +52,7 @@ void testConverter() throws Exception { try (FlinkRowToFlussRowConverter converter = FlinkRowToFlussRowConverter.create(toFlinkRowType(flussRowType))) { InternalRow internalRow = converter.toInternalRow(genRowDataForAllType()); - assertThat(internalRow.getFieldCount()).isEqualTo(22); + assertThat(internalRow.getFieldCount()).isEqualTo(23); assertAllTypeEquals(internalRow); } @@ -61,13 +61,13 @@ void testConverter() throws Exception { FlinkRowToFlussRowConverter.create( toFlinkRowType(flussRowType), KvFormat.COMPACTED)) { InternalRow internalRow = converter.toInternalRow(genRowDataForAllType()); - assertThat(internalRow.getFieldCount()).isEqualTo(22); + assertThat(internalRow.getFieldCount()).isEqualTo(23); assertAllTypeEquals(internalRow); } } private static RowData genRowDataForAllType() { - GenericRowData genericRowData = new GenericRowData(22); + GenericRowData genericRowData = new GenericRowData(23); genericRowData.setField(0, true); genericRowData.setField(1, (byte) 2); genericRowData.setField(2, Short.parseShort("10")); @@ -91,6 +91,8 @@ private static RowData genRowDataForAllType() { genericRowData.setField( 18, TimestampData.fromLocalDateTime(LocalDateTime.parse("2023-10-25T12:01:13.182"))); + + // 19: array genericRowData.setField( 19, new GenericArrayData(new Integer[] {1, 2, 3, 4, 5, -11, null, 444, 102234})); genericRowData.setField( @@ -107,6 +109,11 @@ private static RowData genRowDataForAllType() { new GenericArrayData( new StringData[] {fromString("hello"), fromString("world")}) })); + + // 22: row (nested row with fields: u1: INT, u2: ROW(v1: INT), u3: STRING) + genericRowData.setField( + 22, GenericRowData.of(123, GenericRowData.of(22), StringData.fromString("Test"))); + return genericRowData; } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java index 0c7515f3ae..06e48ffa2b 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlussRowToFlinkRowConverterTest.java @@ -124,6 +124,13 @@ void testConverter() throws Exception { .toObjectArray(DataTypes.STRING()); assertThat(stringArray2) .isEqualTo(new BinaryString[] {fromString("hello"), fromString("world")}); + + // nested row: ROW, u3: STRING> + assertThat(flinkRow.getRow(22, 3)).isNotNull(); + assertThat(flinkRow.getRow(22, 3).getInt(0)).isEqualTo(123); + assertThat(flinkRow.getRow(22, 3).getRow(1, 1)).isNotNull(); + assertThat(flinkRow.getRow(22, 3).getRow(1, 1).getInt(0)).isEqualTo(22); + assertThat(flinkRow.getRow(22, 3).getString(2).toString()).isEqualTo("Test"); } } } diff --git a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java index 9c25c4a4ca..dd76bd2ec2 100644 --- a/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java +++ b/fluss-lake/fluss-lake-iceberg/src/main/java/org/apache/fluss/lake/iceberg/source/IcebergRecordAsFlussRow.java @@ -155,6 +155,8 @@ public InternalArray getArray(int pos) { throw new UnsupportedOperationException(); } - // TODO: Support Map type conversion from Iceberg to Fluss - // TODO: Support Row type conversion from Iceberg to Fluss + @Override + public InternalRow getRow(int pos, int numFields) { + throw new UnsupportedOperationException(); + } } diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java new file mode 100644 index 0000000000..285d41a7ba --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArray.java @@ -0,0 +1,180 @@ +/* + * 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.fluss.lake.paimon.utils; + +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; + +import org.apache.paimon.data.Timestamp; + +/** Wraps a Paimon {@link org.apache.paimon.data.InternalArray} as a Fluss {@link InternalArray}. */ +public class PaimonArrayAsFlussArray implements InternalArray { + + private final org.apache.paimon.data.InternalArray paimonArray; + + public PaimonArrayAsFlussArray(org.apache.paimon.data.InternalArray paimonArray) { + this.paimonArray = paimonArray; + } + + @Override + public int size() { + return paimonArray.size(); + } + + @Override + public boolean isNullAt(int pos) { + return paimonArray.isNullAt(pos); + } + + @Override + public boolean getBoolean(int pos) { + return paimonArray.getBoolean(pos); + } + + @Override + public byte getByte(int pos) { + return paimonArray.getByte(pos); + } + + @Override + public short getShort(int pos) { + return paimonArray.getShort(pos); + } + + @Override + public int getInt(int pos) { + return paimonArray.getInt(pos); + } + + @Override + public long getLong(int pos) { + return paimonArray.getLong(pos); + } + + @Override + public float getFloat(int pos) { + return paimonArray.getFloat(pos); + } + + @Override + public double getDouble(int pos) { + return paimonArray.getDouble(pos); + } + + @Override + public BinaryString getChar(int pos, int length) { + return BinaryString.fromBytes(paimonArray.getString(pos).toBytes()); + } + + @Override + public BinaryString getString(int pos) { + return BinaryString.fromBytes(paimonArray.getString(pos).toBytes()); + } + + @Override + public Decimal getDecimal(int pos, int precision, int scale) { + org.apache.paimon.data.Decimal paimonDecimal = + paimonArray.getDecimal(pos, precision, scale); + if (paimonDecimal.isCompact()) { + return Decimal.fromUnscaledLong(paimonDecimal.toUnscaledLong(), precision, scale); + } else { + return Decimal.fromBigDecimal(paimonDecimal.toBigDecimal(), precision, scale); + } + } + + @Override + public TimestampNtz getTimestampNtz(int pos, int precision) { + Timestamp timestamp = paimonArray.getTimestamp(pos, precision); + if (TimestampNtz.isCompact(precision)) { + return TimestampNtz.fromMillis(timestamp.getMillisecond()); + } else { + return TimestampNtz.fromMillis( + timestamp.getMillisecond(), timestamp.getNanoOfMillisecond()); + } + } + + @Override + public TimestampLtz getTimestampLtz(int pos, int precision) { + Timestamp timestamp = paimonArray.getTimestamp(pos, precision); + if (TimestampLtz.isCompact(precision)) { + return TimestampLtz.fromEpochMillis(timestamp.getMillisecond()); + } else { + return TimestampLtz.fromEpochMillis( + timestamp.getMillisecond(), timestamp.getNanoOfMillisecond()); + } + } + + @Override + public byte[] getBinary(int pos, int length) { + return paimonArray.getBinary(pos); + } + + @Override + public byte[] getBytes(int pos) { + return paimonArray.getBinary(pos); + } + + @Override + public InternalArray getArray(int pos) { + return new PaimonArrayAsFlussArray(paimonArray.getArray(pos)); + } + + @Override + public InternalRow getRow(int pos, int numFields) { + return new PaimonRowAsFlussRow(paimonArray.getRow(pos, numFields)); + } + + @Override + public boolean[] toBooleanArray() { + return paimonArray.toBooleanArray(); + } + + @Override + public byte[] toByteArray() { + return paimonArray.toByteArray(); + } + + @Override + public short[] toShortArray() { + return paimonArray.toShortArray(); + } + + @Override + public int[] toIntArray() { + return paimonArray.toIntArray(); + } + + @Override + public long[] toLongArray() { + return paimonArray.toLongArray(); + } + + @Override + public float[] toFloatArray() { + return paimonArray.toFloatArray(); + } + + @Override + public double[] toDoubleArray() { + return paimonArray.toDoubleArray(); + } +} diff --git a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java index 80896dc7bb..803432ec40 100644 --- a/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java +++ b/fluss-lake/fluss-lake-paimon/src/main/java/org/apache/fluss/lake/paimon/utils/PaimonRowAsFlussRow.java @@ -143,10 +143,17 @@ public byte[] getBytes(int pos) { @Override public InternalArray getArray(int pos) { - // TODO: Support Array type conversion from Paimon to Fluss - throw new UnsupportedOperationException(); + return new PaimonArrayAsFlussArray(paimonRow.getArray(pos)); } - // TODO: Support Map type conversion from Paimon to Fluss - // TODO: Support Row type conversion from Paimon to Fluss + // TODO: Add Map type support in future + // @Override + // public InternalMap getMap(int pos) { + // return new PaimonMapAsFlussMap(paimonRow.getMap(pos)); + // } + + @Override + public InternalRow getRow(int pos, int numFields) { + return new PaimonRowAsFlussRow(paimonRow.getRow(pos, numFields)); + } } diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArrayTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArrayTest.java new file mode 100644 index 0000000000..c1efa2651b --- /dev/null +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/utils/PaimonArrayAsFlussArrayTest.java @@ -0,0 +1,251 @@ +/* + * 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.fluss.lake.paimon.utils; + +import org.apache.fluss.row.BinaryString; +import org.apache.fluss.row.Decimal; +import org.apache.fluss.row.InternalArray; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.TimestampLtz; +import org.apache.fluss.row.TimestampNtz; + +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.Timestamp; +import org.junit.jupiter.api.Test; + +import java.math.BigDecimal; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link PaimonArrayAsFlussArray}. */ +class PaimonArrayAsFlussArrayTest { + + @Test + void testPrimitiveArrayMethods() { + GenericArray paimonArray = + new GenericArray(new Object[] {true, (byte) 1, (short) 2, 3, 4L, 5.5f, 6.6d}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + assertThat(flussArray.size()).isEqualTo(7); + assertThat(flussArray.getBoolean(0)).isTrue(); + assertThat(flussArray.getByte(1)).isEqualTo((byte) 1); + assertThat(flussArray.getShort(2)).isEqualTo((short) 2); + assertThat(flussArray.getInt(3)).isEqualTo(3); + assertThat(flussArray.getLong(4)).isEqualTo(4L); + assertThat(flussArray.getFloat(5)).isEqualTo(5.5f); + assertThat(flussArray.getDouble(6)).isEqualTo(6.6d); + } + + @Test + void testStringAndCharMethods() { + GenericArray paimonArray = + new GenericArray( + new Object[] { + org.apache.paimon.data.BinaryString.fromString("hello"), + org.apache.paimon.data.BinaryString.fromString("world") + }); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + BinaryString str = flussArray.getString(0); + assertThat(str.toString()).isEqualTo("hello"); + + BinaryString charStr = flussArray.getChar(1, 5); + assertThat(charStr.toString()).isEqualTo("world"); + } + + @Test + void testBinaryAndBytesMethods() { + GenericArray paimonArray = + new GenericArray(new Object[] {new byte[] {1, 2, 3}, new byte[] {4, 5, 6}}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + assertThat(flussArray.getBinary(0, 3)).isEqualTo(new byte[] {1, 2, 3}); + assertThat(flussArray.getBytes(1)).isEqualTo(new byte[] {4, 5, 6}); + } + + @Test + void testDecimalCompact() { + org.apache.paimon.data.Decimal paimonDecimal = + org.apache.paimon.data.Decimal.fromUnscaledLong(12345L, 10, 2); + GenericArray paimonArray = new GenericArray(new Object[] {paimonDecimal}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + Decimal decimal = flussArray.getDecimal(0, 10, 2); + assertThat(decimal.toUnscaledLong()).isEqualTo(12345L); + assertThat(decimal.precision()).isEqualTo(10); + assertThat(decimal.scale()).isEqualTo(2); + } + + @Test + void testDecimalNonCompact() { + BigDecimal bigDecimalValue = new BigDecimal("12345678901234567890.12"); + org.apache.paimon.data.Decimal paimonDecimal = + org.apache.paimon.data.Decimal.fromBigDecimal(bigDecimalValue, 25, 2); + GenericArray paimonArray = new GenericArray(new Object[] {paimonDecimal}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + Decimal decimal = flussArray.getDecimal(0, 25, 2); + assertThat(decimal.toBigDecimal()).isEqualTo(bigDecimalValue); + } + + @Test + void testTimestampNtzCompact() { + Timestamp paimonTimestamp = Timestamp.fromEpochMillis(1672531200000L); + GenericArray paimonArray = new GenericArray(new Object[] {paimonTimestamp}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + TimestampNtz ntz = flussArray.getTimestampNtz(0, 3); + assertThat(ntz.getMillisecond()).isEqualTo(1672531200000L); + } + + @Test + void testTimestampNtzNonCompact() { + Timestamp paimonTimestamp = Timestamp.fromEpochMillis(1672531200000L, 123456); + GenericArray paimonArray = new GenericArray(new Object[] {paimonTimestamp}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + TimestampNtz ntz = flussArray.getTimestampNtz(0, 9); + assertThat(ntz.getMillisecond()).isEqualTo(1672531200000L); + assertThat(ntz.getNanoOfMillisecond()).isEqualTo(123456); + } + + @Test + void testTimestampLtzCompact() { + Timestamp paimonTimestamp = Timestamp.fromEpochMillis(1672531200000L); + GenericArray paimonArray = new GenericArray(new Object[] {paimonTimestamp}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + TimestampLtz ltz = flussArray.getTimestampLtz(0, 3); + assertThat(ltz.getEpochMillisecond()).isEqualTo(1672531200000L); + } + + @Test + void testTimestampLtzNonCompact() { + Timestamp paimonTimestamp = Timestamp.fromEpochMillis(1672531200000L, 654321); + GenericArray paimonArray = new GenericArray(new Object[] {paimonTimestamp}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + TimestampLtz ltz = flussArray.getTimestampLtz(0, 9); + assertThat(ltz.getEpochMillisecond()).isEqualTo(1672531200000L); + assertThat(ltz.getNanoOfMillisecond()).isEqualTo(654321); + } + + @Test + void testIsNullAt() { + GenericArray paimonArray = new GenericArray(new Object[] {null, 1, null}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + assertThat(flussArray.isNullAt(0)).isTrue(); + assertThat(flussArray.isNullAt(1)).isFalse(); + assertThat(flussArray.isNullAt(2)).isTrue(); + } + + @Test + void testNestedArray() { + GenericArray innerArray = new GenericArray(new Object[] {1, 2, 3}); + GenericArray outerArray = new GenericArray(new Object[] {innerArray}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(outerArray); + + InternalArray nested = flussArray.getArray(0); + assertThat(nested).isInstanceOf(PaimonArrayAsFlussArray.class); + assertThat(nested.size()).isEqualTo(3); + assertThat(nested.getInt(0)).isEqualTo(1); + assertThat(nested.getInt(1)).isEqualTo(2); + assertThat(nested.getInt(2)).isEqualTo(3); + } + + @Test + void testNestedRow() { + GenericRow innerRow = new GenericRow(2); + innerRow.setField(0, 42); + innerRow.setField(1, org.apache.paimon.data.BinaryString.fromString("test")); + + GenericArray paimonArray = new GenericArray(new Object[] {innerRow}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + InternalRow row = flussArray.getRow(0, 2); + assertThat(row).isInstanceOf(PaimonRowAsFlussRow.class); + assertThat(row.getInt(0)).isEqualTo(42); + assertThat(row.getString(1).toString()).isEqualTo("test"); + } + + @Test + void testToBooleanArray() { + GenericArray paimonArray = new GenericArray(new boolean[] {true, false, true}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + boolean[] result = flussArray.toBooleanArray(); + assertThat(result).containsExactly(true, false, true); + } + + @Test + void testToByteArray() { + GenericArray paimonArray = new GenericArray(new byte[] {1, 2, 3}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + byte[] result = flussArray.toByteArray(); + assertThat(result).containsExactly((byte) 1, (byte) 2, (byte) 3); + } + + @Test + void testToShortArray() { + GenericArray paimonArray = new GenericArray(new short[] {10, 20, 30}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + short[] result = flussArray.toShortArray(); + assertThat(result).containsExactly((short) 10, (short) 20, (short) 30); + } + + @Test + void testToIntArray() { + GenericArray paimonArray = new GenericArray(new int[] {100, 200, 300}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + int[] result = flussArray.toIntArray(); + assertThat(result).containsExactly(100, 200, 300); + } + + @Test + void testToLongArray() { + GenericArray paimonArray = new GenericArray(new long[] {1000L, 2000L, 3000L}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + long[] result = flussArray.toLongArray(); + assertThat(result).containsExactly(1000L, 2000L, 3000L); + } + + @Test + void testToFloatArray() { + GenericArray paimonArray = new GenericArray(new float[] {1.1f, 2.2f, 3.3f}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + float[] result = flussArray.toFloatArray(); + assertThat(result).containsExactly(1.1f, 2.2f, 3.3f); + } + + @Test + void testToDoubleArray() { + GenericArray paimonArray = new GenericArray(new double[] {1.11d, 2.22d, 3.33d}); + PaimonArrayAsFlussArray flussArray = new PaimonArrayAsFlussArray(paimonArray); + + double[] result = flussArray.toDoubleArray(); + assertThat(result).containsExactly(1.11d, 2.22d, 3.33d); + } +} diff --git a/fluss-test-coverage/pom.xml b/fluss-test-coverage/pom.xml index 46c34da57a..f82aa3321f 100644 --- a/fluss-test-coverage/pom.xml +++ b/fluss-test-coverage/pom.xml @@ -325,6 +325,8 @@ org.apache.fluss.security.auth.ServerAuthenticator org.apache.fluss.config.cluster.AlterConfig + + org.apache.fluss.shaded.arrow.org.apache.arrow.vector.util.* org.apache.fluss.flink.utils.* org.apache.fluss.flink.source.*