From c4e970dc3aa43ebe10d0352dc90d8919ac9071ec Mon Sep 17 00:00:00 2001 From: Nathaniel Bauernfeind Date: Sun, 15 Sep 2024 23:29:08 -0600 Subject: [PATCH] Barrage: Refactor Read/Write Chunk Factories --- .../java/io/deephaven/chunk/BooleanChunk.java | 5 + .../java/io/deephaven/chunk/ByteChunk.java | 5 + .../java/io/deephaven/chunk/CharChunk.java | 5 + .../main/java/io/deephaven/chunk/Chunk.java | 6 + .../java/io/deephaven/chunk/DoubleChunk.java | 5 + .../java/io/deephaven/chunk/FloatChunk.java | 5 + .../java/io/deephaven/chunk/IntChunk.java | 5 + .../java/io/deephaven/chunk/LongChunk.java | 5 + .../java/io/deephaven/chunk/ObjectChunk.java | 5 + .../java/io/deephaven/chunk/ShortChunk.java | 5 + .../engine/table/impl/QueryTable.java | 1 + .../table/impl/preview/ArrayPreview.java | 5 +- .../table/impl/sources/ReinterpretUtils.java | 1 + ...nerator.java => BarrageMessageWriter.java} | 33 +- ...mpl.java => BarrageMessageWriterImpl.java} | 328 ++--- .../barrage/BarrageSnapshotOptions.java | 4 +- .../barrage/BarrageSubscriptionOptions.java | 4 +- .../ChunkListInputStreamGenerator.java | 56 - .../extensions/barrage/ChunkListWriter.java | 54 + .../chunk/BaseChunkInputStreamGenerator.java | 134 -- .../barrage/chunk/BaseChunkReader.java | 38 + .../barrage/chunk/BaseChunkWriter.java | 213 +++ .../barrage/chunk/BooleanChunkReader.java | 20 +- ...Generator.java => BooleanChunkWriter.java} | 88 +- .../chunk/ByteChunkInputStreamGenerator.java | 161 --- .../barrage/chunk/ByteChunkReader.java | 65 +- .../barrage/chunk/ByteChunkWriter.java | 101 ++ .../chunk/CharChunkInputStreamGenerator.java | 157 --- .../barrage/chunk/CharChunkReader.java | 65 +- .../barrage/chunk/CharChunkWriter.java | 97 ++ .../chunk/ChunkInputStreamGenerator.java | 117 -- .../extensions/barrage/chunk/ChunkReader.java | 122 +- .../extensions/barrage/chunk/ChunkWriter.java | 171 +++ ...faultChunkInputStreamGeneratorFactory.java | 178 --- .../chunk/DefaultChunkReaderFactory.java | 1244 +++++++++++++++++ .../chunk/DefaultChunkReadingFactory.java | 202 --- .../chunk/DefaultChunkWriterFactory.java | 1240 ++++++++++++++++ .../DoubleChunkInputStreamGenerator.java | 162 --- .../barrage/chunk/DoubleChunkReader.java | 174 ++- .../barrage/chunk/DoubleChunkWriter.java | 101 ++ .../barrage/chunk/ExpansionKernel.java | 103 ++ ...erator.java => FixedWidthChunkReader.java} | 62 +- .../barrage/chunk/FixedWidthChunkWriter.java | 100 ++ .../chunk/FloatChunkInputStreamGenerator.java | 161 --- .../barrage/chunk/FloatChunkReader.java | 174 ++- .../barrage/chunk/FloatChunkWriter.java | 101 ++ .../chunk/IntChunkInputStreamGenerator.java | 162 --- .../barrage/chunk/IntChunkReader.java | 65 +- .../barrage/chunk/IntChunkWriter.java | 101 ++ .../barrage/chunk/ListChunkReader.java | 149 ++ .../barrage/chunk/ListChunkWriter.java | 247 ++++ .../chunk/LongChunkInputStreamGenerator.java | 162 --- .../barrage/chunk/LongChunkReader.java | 65 +- .../barrage/chunk/LongChunkWriter.java | 101 ++ .../barrage/chunk/NullChunkReader.java | 47 + .../barrage/chunk/NullChunkWriter.java | 52 + .../chunk/ShortChunkInputStreamGenerator.java | 161 --- .../barrage/chunk/ShortChunkReader.java | 65 +- .../barrage/chunk/ShortChunkWriter.java | 101 ++ ...ava => SingleElementListHeaderWriter.java} | 11 +- .../chunk/TransformingChunkReader.java | 69 + .../barrage/chunk/VarBinaryChunkReader.java | 136 ++ ...nerator.java => VarBinaryChunkWriter.java} | 483 +++---- .../VarListChunkInputStreamGenerator.java | 235 ---- .../barrage/chunk/VarListChunkReader.java | 116 -- .../VectorChunkInputStreamGenerator.java | 227 --- .../barrage/chunk/VectorChunkReader.java | 112 -- .../chunk/array/ArrayExpansionKernel.java | 72 +- .../array/BooleanArrayExpansionKernel.java | 64 +- .../BoxedBooleanArrayExpansionKernel.java | 64 +- .../chunk/array/ByteArrayExpansionKernel.java | 71 +- .../chunk/array/CharArrayExpansionKernel.java | 71 +- .../array/DoubleArrayExpansionKernel.java | 71 +- .../array/FloatArrayExpansionKernel.java | 71 +- .../chunk/array/IntArrayExpansionKernel.java | 71 +- .../chunk/array/LongArrayExpansionKernel.java | 71 +- .../array/ObjectArrayExpansionKernel.java | 74 +- .../array/ShortArrayExpansionKernel.java | 71 +- .../vector/ByteVectorExpansionKernel.java | 60 +- .../vector/CharVectorExpansionKernel.java | 60 +- .../vector/DoubleVectorExpansionKernel.java | 60 +- .../vector/FloatVectorExpansionKernel.java | 60 +- .../vector/IntVectorExpansionKernel.java | 60 +- .../vector/LongVectorExpansionKernel.java | 60 +- .../vector/ObjectVectorExpansionKernel.java | 61 +- .../vector/ShortVectorExpansionKernel.java | 60 +- .../chunk/vector/VectorExpansionKernel.java | 60 +- .../barrage/table/BarrageRedirectedTable.java | 1 - .../barrage/table/BarrageTable.java | 21 +- .../barrage/util/ArrowToTableConverter.java | 24 +- ...mReader.java => BarrageMessageReader.java} | 13 +- ...der.java => BarrageMessageReaderImpl.java} | 31 +- .../barrage/util/BarrageProtoUtil.java | 5 - .../extensions/barrage/util/BarrageUtil.java | 135 +- .../extensions/barrage/util/Float16.java | 168 +++ .../barrage/util/StreamReaderOptions.java | 31 - .../barrage/util/TableToArrowConverter.java | 10 +- .../extensions/barrage/Barrage.gwt.xml | 8 +- ...Test.java => BarrageStreamWriterTest.java} | 2 +- .../chunk/BarrageColumnRoundTripTest.java | 95 +- .../client/impl/BarrageSnapshotImpl.java | 8 +- .../client/impl/BarrageSubscriptionImpl.java | 8 +- .../replicators/ReplicateBarrageUtils.java | 9 +- .../ReplicateSourcesAndChunks.java | 3 +- .../server/arrow/ArrowFlightUtil.java | 16 +- .../deephaven/server/arrow/ArrowModule.java | 8 +- .../server/arrow/FlightServiceGrpcImpl.java | 6 +- .../barrage/BarrageMessageProducer.java | 71 +- .../HierarchicalTableViewSubscription.java | 37 +- .../server/barrage/BarrageBlinkTableTest.java | 8 +- .../barrage/BarrageMessageRoundTripTest.java | 12 +- .../test/FlightMessageRoundTripTest.java | 2 +- ...ader.java => WebBarrageMessageReader.java} | 19 +- .../api/barrage/WebChunkReaderFactory.java | 65 +- .../AbstractTableSubscription.java | 4 +- .../TableViewportSubscription.java | 4 +- 116 files changed, 6757 insertions(+), 4334 deletions(-) rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/{BarrageStreamGenerator.java => BarrageMessageWriter.java} (69%) rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/{BarrageStreamGeneratorImpl.java => BarrageMessageWriterImpl.java} (76%) delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkReader.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkWriter.java rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/{BooleanChunkInputStreamGenerator.java => BooleanChunkWriter.java} (51%) delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkInputStreamGeneratorFactory.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReaderFactory.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReadingFactory.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkWriterFactory.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkWriter.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ExpansionKernel.java rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/{FixedWidthChunkInputStreamGenerator.java => FixedWidthChunkReader.java} (71%) create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkWriter.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkReader.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkWriter.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkReader.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkWriter.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkWriter.java rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/{SingleElementListHeaderInputStreamGenerator.java => SingleElementListHeaderWriter.java} (76%) create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/TransformingChunkReader.java create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkReader.java rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/{VarBinaryChunkInputStreamGenerator.java => VarBinaryChunkWriter.java} (51%) delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkReader.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java delete mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkReader.java rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/{StreamReader.java => BarrageMessageReader.java} (67%) rename extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/{BarrageStreamReader.java => BarrageMessageReaderImpl.java} (94%) create mode 100644 extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/Float16.java rename extensions/barrage/src/test/java/io/deephaven/extensions/barrage/{BarrageStreamGeneratorTest.java => BarrageStreamWriterTest.java} (97%) rename web/client-api/src/main/java/io/deephaven/web/client/api/barrage/{WebBarrageStreamReader.java => WebBarrageMessageReader.java} (94%) diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/BooleanChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/BooleanChunk.java index 063ba8c8a70..fa29e8d40eb 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/BooleanChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/BooleanChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -74,6 +75,10 @@ public final boolean get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_BOOLEAN; + } + @Override public BooleanChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/ByteChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/ByteChunk.java index f4988ae2ddd..92e58b73909 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/ByteChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/ByteChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -78,6 +79,10 @@ public final byte get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_BYTE; + } + @Override public ByteChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/CharChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/CharChunk.java index 3671c48a223..97d04681f91 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/CharChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/CharChunk.java @@ -3,6 +3,7 @@ // package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -73,6 +74,10 @@ public final char get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_CHAR; + } + @Override public CharChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/Chunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/Chunk.java index 35e152dcd0f..7d9c5e03605 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/Chunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/Chunk.java @@ -109,6 +109,12 @@ default void copyToBuffer(int srcOffset, @NotNull Buffer destBuffer, int destOff */ int size(); + /** + * @return whether The value offset is null + * @param index The index to check + */ + boolean isNullAt(int index); + /** * @return The underlying chunk type */ diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/DoubleChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/DoubleChunk.java index 640a7c0a020..b53a08921ef 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/DoubleChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/DoubleChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -77,6 +78,10 @@ public final double get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_DOUBLE; + } + @Override public DoubleChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/FloatChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/FloatChunk.java index a30f212ee1b..806adf6e10e 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/FloatChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/FloatChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -77,6 +78,10 @@ public final float get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_FLOAT; + } + @Override public FloatChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/IntChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/IntChunk.java index 7f615adec8b..c5c46e591e6 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/IntChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/IntChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -77,6 +78,10 @@ public final int get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_INT; + } + @Override public IntChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/LongChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/LongChunk.java index 1486e20bbd7..9b34855dfc3 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/LongChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/LongChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -77,6 +78,10 @@ public final long get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_LONG; + } + @Override public LongChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/ObjectChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/ObjectChunk.java index f89c3727ae4..49ac3556670 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/ObjectChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/ObjectChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -77,6 +78,10 @@ public final T get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == null; + } + @Override public ObjectChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/ShortChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/ShortChunk.java index 7d99a61b546..12cb89b260c 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/ShortChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/ShortChunk.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.chunk; +import io.deephaven.util.QueryConstants; import io.deephaven.util.type.ArrayTypeUtils; import io.deephaven.chunk.attributes.Any; @@ -77,6 +78,10 @@ public final short get(int index) { return data[offset + index]; } + public final boolean isNullAt(int index) { + return data[offset + index] == QueryConstants.NULL_SHORT; + } + @Override public ShortChunk slice(int offset, int capacity) { ChunkHelpers.checkSliceArgs(size, offset, capacity); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index 1c227ea3ae7..6629db9ab2a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -2613,6 +2613,7 @@ private Table snapshotIncrementalInternal(final Table base, final boolean doInit new ListenerRecorder("snapshotIncremental (triggerTable)", this, resultTable); addUpdateListener(triggerListenerRecorder); + dropColumns(getColumnSourceMap().keySet()); final SnapshotIncrementalListener listener = new SnapshotIncrementalListener(this, resultTable, resultColumns, baseListenerRecorder, triggerListenerRecorder, baseTable, triggerColumns); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/preview/ArrayPreview.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/preview/ArrayPreview.java index d1c742a7ddc..82514bfd29f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/preview/ArrayPreview.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/preview/ArrayPreview.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.preview; +import io.deephaven.util.type.TypeUtils; import io.deephaven.vector.Vector; import io.deephaven.vector.VectorFactory; import org.jetbrains.annotations.NotNull; @@ -34,7 +35,9 @@ public static ArrayPreview fromArray(final Object array) { if (componentType == boolean.class) { return new ArrayPreview(convertToString((boolean[]) array)); } - return new ArrayPreview(VectorFactory.forElementType(componentType) + // Boxed primitives need the Object wrapper. + final Class elementType = TypeUtils.isBoxedType(componentType) ? Object.class : componentType; + return new ArrayPreview(VectorFactory.forElementType(elementType) .vectorWrap(array) .toString(ARRAY_SIZE_CUTOFF)); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java index baf4e22309c..5b2ab07a8b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ReinterpretUtils.java @@ -283,6 +283,7 @@ public static Class maybeConvertToPrimitiveDataType(@NotNull final Class d return byte.class; } if (dataType == Instant.class || dataType == ZonedDateTime.class) { + // Note: not all ZonedDateTime sources are convertible to long, so this doesn't match column source behavior return long.class; } return dataType; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriter.java similarity index 69% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageStreamGenerator.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriter.java index 2c0375235ae..a99b41205df 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriter.java @@ -4,8 +4,11 @@ package io.deephaven.extensions.barrage; import com.google.flatbuffers.FlatBufferBuilder; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.util.BarrageMessage; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; import io.deephaven.extensions.barrage.util.DefensiveDrainable; import io.deephaven.util.SafeCloseable; import org.jetbrains.annotations.NotNull; @@ -17,10 +20,10 @@ import java.util.function.ToIntFunction; /** - * A StreamGenerator takes a BarrageMessage and re-uses portions of the serialized payload across different subscribers - * that may subscribe to different viewports and columns. + * A {@code BarrageMessageWriter} takes a {@link BarrageMessage} and re-uses portions of the serialized payload across + * different subscribers that may subscribe to different viewports and columns. */ -public interface BarrageStreamGenerator extends SafeCloseable { +public interface BarrageMessageWriter extends SafeCloseable { /** * Represents a single update, which might be sent as multiple distinct payloads as necessary based in the @@ -32,16 +35,18 @@ interface MessageView { interface Factory { /** - * Create a StreamGenerator that now owns the BarrageMessage. + * Create a {@code BarrageMessageWriter} that now owns the {@link BarrageMessage}. * * @param message the message that contains the update that we would like to propagate * @param metricsConsumer a method that can be used to record write metrics */ - BarrageStreamGenerator newGenerator( - BarrageMessage message, BarragePerformanceLog.WriteMetricsConsumer metricsConsumer); + BarrageMessageWriter newMessageWriter( + @NotNull BarrageMessage message, + @NotNull ChunkWriter>[] chunkWriters, + @NotNull BarragePerformanceLog.WriteMetricsConsumer metricsConsumer); /** - * Create a MessageView of the Schema to send as the initial message to a new subscriber. + * Create a {@link MessageView} of the Schema to send as the initial message to a new subscriber. * * @param schemaPayloadWriter a function that writes schema data to a {@link FlatBufferBuilder} and returns the * schema offset @@ -51,21 +56,22 @@ BarrageStreamGenerator newGenerator( } /** - * @return the BarrageMessage that this generator is operating on + * @return the {@link BarrageMessage} that this writer is operating on */ BarrageMessage getMessage(); /** - * Obtain a Full-Subscription View of this StreamGenerator that can be sent to a single subscriber. + * Obtain a Full-Subscription {@link MessageView} of this {@code BarrageMessageWriter} that can be sent to a single + * subscriber. * * @param options serialization options for this specific view - * @param isInitialSnapshot indicates whether or not this is the first snapshot for the listener + * @param isInitialSnapshot indicates whether this is the first snapshot for the listener * @return a MessageView filtered by the subscription properties that can be sent to that subscriber */ MessageView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnapshot); /** - * Obtain a View of this StreamGenerator that can be sent to a single subscriber. + * Obtain a {@link MessageView} of this {@code BarrageMessageWriter} that can be sent to a single subscriber. * * @param options serialization options for this specific view * @param isInitialSnapshot indicates whether or not this is the first snapshot for the listener @@ -79,7 +85,8 @@ MessageView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnap boolean reverseViewport, @Nullable RowSet keyspaceViewport, BitSet subscribedColumns); /** - * Obtain a Full-Snapshot View of this StreamGenerator that can be sent to a single requestor. + * Obtain a Full-Snapshot {@link MessageView} of this {@code BarrageMessageWriter} that can be sent to a single + * requestor. * * @param options serialization options for this specific view * @return a MessageView filtered by the snapshot properties that can be sent to that requestor @@ -87,7 +94,7 @@ MessageView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnap MessageView getSnapshotView(BarrageSnapshotOptions options); /** - * Obtain a View of this StreamGenerator that can be sent to a single requestor. + * Obtain a {@link MessageView} of this {@code BarrageMessageWriter} that can be sent to a single requestor. * * @param options serialization options for this specific view * @param viewport is the position-space viewport diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageStreamGeneratorImpl.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java similarity index 76% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageStreamGeneratorImpl.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java index c63a527104b..aad8b0e242e 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageStreamGeneratorImpl.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java @@ -14,6 +14,7 @@ import io.deephaven.barrage.flatbuf.BarrageMessageWrapper; import io.deephaven.barrage.flatbuf.BarrageModColumnMetadata; import io.deephaven.barrage.flatbuf.BarrageUpdateMetadata; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableLongChunk; @@ -24,13 +25,12 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.impl.ExternalizableRowSetUtils; import io.deephaven.engine.table.impl.util.BarrageMessage; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator; -import io.deephaven.extensions.barrage.chunk.DefaultChunkInputStreamGeneratorFactory; -import io.deephaven.extensions.barrage.chunk.SingleElementListHeaderInputStreamGenerator; +import io.deephaven.extensions.barrage.chunk.ChunkReader; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; +import io.deephaven.extensions.barrage.chunk.SingleElementListHeaderWriter; import io.deephaven.extensions.barrage.util.ExposedByteArrayOutputStream; import io.deephaven.extensions.barrage.util.BarrageUtil; import io.deephaven.extensions.barrage.util.DefensiveDrainable; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; import io.deephaven.proto.flight.util.MessageHelper; @@ -52,41 +52,43 @@ import java.util.function.Consumer; import java.util.function.ToIntFunction; -import static io.deephaven.extensions.barrage.chunk.BaseChunkInputStreamGenerator.PADDING_BUFFER; +import static io.deephaven.extensions.barrage.chunk.BaseChunkWriter.PADDING_BUFFER; import static io.deephaven.proto.flight.util.MessageHelper.toIpcBytes; -public class BarrageStreamGeneratorImpl implements BarrageStreamGenerator { +public class BarrageMessageWriterImpl implements BarrageMessageWriter { - private static final Logger log = LoggerFactory.getLogger(BarrageStreamGeneratorImpl.class); + private static final Logger log = LoggerFactory.getLogger(BarrageMessageWriterImpl.class); // NB: This should likely be something smaller, such as 1<<16, but since the js api is not yet able // to receive multiple record batches we crank this up to MAX_INT. private static final int DEFAULT_BATCH_SIZE = Configuration.getInstance() - .getIntegerForClassWithDefault(BarrageStreamGeneratorImpl.class, "batchSize", Integer.MAX_VALUE); + .getIntegerForClassWithDefault(BarrageMessageWriterImpl.class, "batchSize", Integer.MAX_VALUE); // defaults to a small value that is likely to succeed and provide data for following batches private static final int DEFAULT_INITIAL_BATCH_SIZE = Configuration.getInstance() - .getIntegerForClassWithDefault(BarrageStreamGeneratorImpl.class, "initialBatchSize", 4096); + .getIntegerForClassWithDefault(BarrageMessageWriterImpl.class, "initialBatchSize", 4096); // default to 100MB to match 100MB java-client and w2w default incoming limits private static final int DEFAULT_MESSAGE_SIZE_LIMIT = Configuration.getInstance() - .getIntegerForClassWithDefault(BarrageStreamGeneratorImpl.class, "maxOutboundMessageSize", + .getIntegerForClassWithDefault(BarrageMessageWriterImpl.class, "maxOutboundMessageSize", 100 * 1024 * 1024); public interface RecordBatchMessageView extends MessageView { boolean isViewport(); - StreamReaderOptions options(); + ChunkReader.Options options(); RowSet addRowOffsets(); RowSet modRowOffsets(int col); } - public static class Factory implements BarrageStreamGenerator.Factory { + public static class Factory implements BarrageMessageWriter.Factory { @Override - public BarrageStreamGenerator newGenerator( - final BarrageMessage message, final BarragePerformanceLog.WriteMetricsConsumer metricsConsumer) { - return new BarrageStreamGeneratorImpl(message, metricsConsumer); + public BarrageMessageWriter newMessageWriter( + @NotNull final BarrageMessage message, + @NotNull final ChunkWriter>[] chunkWriters, + @NotNull final BarragePerformanceLog.WriteMetricsConsumer metricsConsumer) { + return new BarrageMessageWriterImpl(message, chunkWriters, metricsConsumer); } @Override @@ -104,9 +106,11 @@ public MessageView getSchemaView(@NotNull final ToIntFunction */ public static class ArrowFactory extends Factory { @Override - public BarrageStreamGenerator newGenerator( - BarrageMessage message, BarragePerformanceLog.WriteMetricsConsumer metricsConsumer) { - return new BarrageStreamGeneratorImpl(message, metricsConsumer) { + public BarrageMessageWriter newMessageWriter( + @NotNull final BarrageMessage message, + @NotNull final ChunkWriter>[] chunkWriters, + @NotNull final BarragePerformanceLog.WriteMetricsConsumer metricsConsumer) { + return new BarrageMessageWriterImpl(message, chunkWriters, metricsConsumer) { @Override protected void writeHeader( ByteBuffer metadata, @@ -119,20 +123,20 @@ protected void writeHeader( } } - public static class ModColumnGenerator implements SafeCloseable { - private final RowSetGenerator rowsModified; - private final ChunkListInputStreamGenerator data; + public static class ModColumnWriter implements SafeCloseable { + private final RowSetWriter rowsModified; + private final ChunkListWriter> chunkListWriter; - ModColumnGenerator(ChunkInputStreamGenerator.Factory factory, final BarrageMessage.ModColumnData col) + ModColumnWriter(final ChunkWriter> writer, final BarrageMessage.ModColumnData col) throws IOException { - rowsModified = new RowSetGenerator(col.rowsModified); - data = new ChunkListInputStreamGenerator(factory, col.type, col.componentType, col.data, col.chunkType); + rowsModified = new RowSetWriter(col.rowsModified); + chunkListWriter = new ChunkListWriter<>(writer, col.data); } @Override public void close() { rowsModified.close(); - data.close(); + chunkListWriter.close(); } } @@ -144,22 +148,25 @@ public void close() { private final boolean isSnapshot; - private final RowSetGenerator rowsAdded; - private final RowSetGenerator rowsIncluded; - private final RowSetGenerator rowsRemoved; - private final RowSetShiftDataGenerator shifted; + private final RowSetWriter rowsAdded; + private final RowSetWriter rowsIncluded; + private final RowSetWriter rowsRemoved; + private final RowSetShiftDataWriter shifted; - private final ChunkListInputStreamGenerator[] addColumnData; - private final ModColumnGenerator[] modColumnData; + private final ChunkListWriter>[] addColumnData; + private final ModColumnWriter[] modColumnData; /** - * Create a barrage stream generator that can slice and dice the barrage message for delivery to clients. + * Create a barrage stream writer that can slice and dice the barrage message for delivery to clients. * - * @param message the generator takes ownership of the message and its internal objects + * @param message the writer takes ownership of the message and its internal objects + * @param chunkWriters the chunk chunkWriters * @param writeConsumer a method that can be used to record write time */ - public BarrageStreamGeneratorImpl(final BarrageMessage message, - final BarragePerformanceLog.WriteMetricsConsumer writeConsumer) { + public BarrageMessageWriterImpl( + @NotNull final BarrageMessage message, + @NotNull final ChunkWriter>[] chunkWriters, + @NotNull final BarragePerformanceLog.WriteMetricsConsumer writeConsumer) { this.message = message; this.writeConsumer = writeConsumer; try { @@ -167,23 +174,23 @@ public BarrageStreamGeneratorImpl(final BarrageMessage message, lastSeq = message.lastSeq; isSnapshot = message.isSnapshot; - rowsAdded = new RowSetGenerator(message.rowsAdded); - rowsIncluded = new RowSetGenerator(message.rowsIncluded); - rowsRemoved = new RowSetGenerator(message.rowsRemoved); - shifted = new RowSetShiftDataGenerator(message.shifted); + rowsAdded = new RowSetWriter(message.rowsAdded); + rowsIncluded = new RowSetWriter(message.rowsIncluded); + rowsRemoved = new RowSetWriter(message.rowsRemoved); + shifted = new RowSetShiftDataWriter(message.shifted); - addColumnData = new ChunkListInputStreamGenerator[message.addColumnData.length]; + // noinspection unchecked + addColumnData = (ChunkListWriter>[]) new ChunkListWriter[message.addColumnData.length]; for (int i = 0; i < message.addColumnData.length; ++i) { BarrageMessage.AddColumnData columnData = message.addColumnData[i]; - addColumnData[i] = new ChunkListInputStreamGenerator(DefaultChunkInputStreamGeneratorFactory.INSTANCE, - columnData.type, columnData.componentType, - columnData.data, columnData.chunkType); + // noinspection resource + addColumnData[i] = new ChunkListWriter<>(chunkWriters[i], columnData.data); } - modColumnData = new ModColumnGenerator[message.modColumnData.length]; + modColumnData = new ModColumnWriter[message.modColumnData.length]; for (int i = 0; i < modColumnData.length; ++i) { - modColumnData[i] = new ModColumnGenerator(DefaultChunkInputStreamGeneratorFactory.INSTANCE, - message.modColumnData[i]); + // noinspection resource + modColumnData[i] = new ModColumnWriter(chunkWriters[i], message.modColumnData[i]); } } catch (final IOException e) { throw new UncheckedDeephavenException("unexpected IOException while creating barrage message stream", e); @@ -213,19 +220,9 @@ public void close() { } } - /** - * Obtain a View of this StreamGenerator that can be sent to a single subscriber. - * - * @param options serialization options for this specific view - * @param isInitialSnapshot indicates whether or not this is the first snapshot for the listener - * @param viewport is the position-space viewport - * @param reverseViewport is the viewport reversed (relative to end of table instead of beginning) - * @param keyspaceViewport is the key-space viewport - * @param subscribedColumns are the columns subscribed for this view - * @return a MessageView filtered by the subscription properties that can be sent to that subscriber - */ @Override - public MessageView getSubView(final BarrageSubscriptionOptions options, + public MessageView getSubView( + final BarrageSubscriptionOptions options, final boolean isInitialSnapshot, @Nullable final RowSet viewport, final boolean reverseViewport, @@ -235,13 +232,6 @@ public MessageView getSubView(final BarrageSubscriptionOptions options, subscribedColumns); } - /** - * Obtain a Full-Subscription View of this StreamGenerator that can be sent to a single subscriber. - * - * @param options serialization options for this specific view - * @param isInitialSnapshot indicates whether or not this is the first snapshot for the listener - * @return a MessageView filtered by the subscription properties that can be sent to that subscriber - */ @Override public MessageView getSubView(BarrageSubscriptionOptions options, boolean isInitialSnapshot) { return getSubView(options, isInitialSnapshot, null, false, null, null); @@ -282,7 +272,7 @@ public SubView(final BarrageSubscriptionOptions options, // precompute the modified column indexes, and calculate total rows needed long numModRows = 0; for (int ii = 0; ii < modColumnData.length; ++ii) { - final ModColumnGenerator mcd = modColumnData[ii]; + final ModColumnWriter mcd = modColumnData[ii]; if (keyspaceViewport != null) { try (WritableRowSet intersect = keyspaceViewport.intersect(mcd.rowsModified.original)) { @@ -324,7 +314,7 @@ public void forEachStream(Consumer visitor) throws IOExcepti if (numAddRows == 0 && numModRows == 0) { // we still need to send a message containing metadata when there are no rows final DefensiveDrainable is = getInputStream(this, 0, 0, actualBatchSize, metadata, - BarrageStreamGeneratorImpl.this::appendAddColumns); + BarrageMessageWriterImpl.this::appendAddColumns); bytesWritten.add(is.available()); visitor.accept(is); writeConsumer.onWrite(bytesWritten.get(), System.nanoTime() - startTm); @@ -333,11 +323,11 @@ public void forEachStream(Consumer visitor) throws IOExcepti // send the add batches (if any) processBatches(visitor, this, numAddRows, maxBatchSize, metadata, - BarrageStreamGeneratorImpl.this::appendAddColumns, bytesWritten); + BarrageMessageWriterImpl.this::appendAddColumns, bytesWritten); // send the mod batches (if any) but don't send metadata twice processBatches(visitor, this, numModRows, maxBatchSize, numAddRows > 0 ? null : metadata, - BarrageStreamGeneratorImpl.this::appendModColumns, bytesWritten); + BarrageMessageWriterImpl.this::appendModColumns, bytesWritten); // clean up the helper indexes addRowOffsets.close(); @@ -364,7 +354,7 @@ public boolean isViewport() { } @Override - public StreamReaderOptions options() { + public ChunkReader.Options options() { return options; } @@ -386,20 +376,20 @@ private ByteBuffer getSubscriptionMetadata() throws IOException { int effectiveViewportOffset = 0; if (isSnapshot && isViewport()) { - try (final RowSetGenerator viewportGen = new RowSetGenerator(viewport)) { + try (final RowSetWriter viewportGen = new RowSetWriter(viewport)) { effectiveViewportOffset = viewportGen.addToFlatBuffer(metadata); } } int effectiveColumnSetOffset = 0; if (isSnapshot && subscribedColumns != null) { - effectiveColumnSetOffset = new BitSetGenerator(subscribedColumns).addToFlatBuffer(metadata); + effectiveColumnSetOffset = new BitSetWriter(subscribedColumns).addToFlatBuffer(metadata); } final int rowsAddedOffset; if (isSnapshot && !isInitialSnapshot) { // client's don't need/want to receive the full RowSet on every snapshot - rowsAddedOffset = EmptyRowSetGenerator.INSTANCE.addToFlatBuffer(metadata); + rowsAddedOffset = EmptyRowSetWriter.INSTANCE.addToFlatBuffer(metadata); } else { rowsAddedOffset = rowsAdded.addToFlatBuffer(metadata); } @@ -417,7 +407,7 @@ private ByteBuffer getSubscriptionMetadata() throws IOException { // now add mod-column streams, and write the mod column indexes TIntArrayList modOffsets = new TIntArrayList(modColumnData.length); - for (final ModColumnGenerator mcd : modColumnData) { + for (final ModColumnWriter mcd : modColumnData) { final int myModRowOffset; if (keyspaceViewport != null) { myModRowOffset = mcd.rowsModified.addToFlatBuffer(keyspaceViewport, metadata); @@ -460,16 +450,6 @@ private ByteBuffer getSubscriptionMetadata() throws IOException { } } - /** - * Obtain a View of this StreamGenerator that can be sent to a single snapshot requestor. - * - * @param options serialization options for this specific view - * @param viewport is the position-space viewport - * @param reverseViewport is the viewport reversed (relative to end of table instead of beginning) - * @param keyspaceViewport is the key-space viewport - * @param snapshotColumns are the columns subscribed for this view - * @return a MessageView filtered by the snapshot properties that can be sent to that subscriber - */ @Override public MessageView getSnapshotView(final BarrageSnapshotOptions options, @Nullable final RowSet viewport, @@ -479,12 +459,6 @@ public MessageView getSnapshotView(final BarrageSnapshotOptions options, return new SnapshotView(options, viewport, reverseViewport, keyspaceViewport, snapshotColumns); } - /** - * Obtain a Full-Snapshot View of this StreamGenerator that can be sent to a single snapshot requestor. - * - * @param options serialization options for this specific view - * @return a MessageView filtered by the snapshot properties that can be sent to that subscriber - */ @Override public MessageView getSnapshotView(BarrageSnapshotOptions options) { return getSnapshotView(options, null, false, null, null); @@ -534,11 +508,11 @@ public void forEachStream(Consumer visitor) throws IOExcepti if (numAddRows == 0) { // we still need to send a message containing metadata when there are no rows visitor.accept(getInputStream(this, 0, 0, actualBatchSize, metadata, - BarrageStreamGeneratorImpl.this::appendAddColumns)); + BarrageMessageWriterImpl.this::appendAddColumns)); } else { // send the add batches processBatches(visitor, this, numAddRows, maxBatchSize, metadata, - BarrageStreamGeneratorImpl.this::appendAddColumns, bytesWritten); + BarrageMessageWriterImpl.this::appendAddColumns, bytesWritten); } addRowOffsets.close(); addRowKeys.close(); @@ -559,7 +533,7 @@ public boolean isViewport() { } @Override - public StreamReaderOptions options() { + public ChunkReader.Options options() { return options; } @@ -578,14 +552,14 @@ private ByteBuffer getSnapshotMetadata() throws IOException { int effectiveViewportOffset = 0; if (isViewport()) { - try (final RowSetGenerator viewportGen = new RowSetGenerator(viewport)) { + try (final RowSetWriter viewportGen = new RowSetWriter(viewport)) { effectiveViewportOffset = viewportGen.addToFlatBuffer(metadata); } } int effectiveColumnSetOffset = 0; if (subscribedColumns != null) { - effectiveColumnSetOffset = new BitSetGenerator(subscribedColumns).addToFlatBuffer(metadata); + effectiveColumnSetOffset = new BitSetWriter(subscribedColumns).addToFlatBuffer(metadata); } final int rowsAddedOffset = rowsAdded.addToFlatBuffer(metadata); @@ -646,8 +620,8 @@ public void forEachStream(Consumer visitor) { private interface ColumnVisitor { int visit(final RecordBatchMessageView view, final long startRange, final int targetBatchSize, final Consumer addStream, - final ChunkInputStreamGenerator.FieldNodeListener fieldNodeListener, - final ChunkInputStreamGenerator.BufferListener bufferListener) throws IOException; + final ChunkWriter.FieldNodeListener fieldNodeListener, + final ChunkWriter.BufferListener bufferListener) throws IOException; } /** @@ -706,14 +680,14 @@ private DefensiveDrainable getInputStream(final RecordBatchMessageView view, fin bufferInfos.get().setSize(0); final MutableLong totalBufferLength = new MutableLong(); - final ChunkInputStreamGenerator.FieldNodeListener fieldNodeListener = + final ChunkWriter.FieldNodeListener fieldNodeListener = (numElements, nullCount) -> { nodeOffsets.ensureCapacityPreserve(nodeOffsets.get().size() + 1); nodeOffsets.get().asWritableObjectChunk() - .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount)); + .add(new ChunkWriter.FieldNodeInfo(numElements, nullCount)); }; - final ChunkInputStreamGenerator.BufferListener bufferListener = (length) -> { + final ChunkWriter.BufferListener bufferListener = (length) -> { totalBufferLength.add(length); bufferInfos.ensureCapacityPreserve(bufferInfos.get().size() + 1); bufferInfos.get().add(length); @@ -725,8 +699,8 @@ private DefensiveDrainable getInputStream(final RecordBatchMessageView view, fin final WritableChunk noChunk = nodeOffsets.get(); RecordBatch.startNodesVector(header, noChunk.size()); for (int i = noChunk.size() - 1; i >= 0; --i) { - final ChunkInputStreamGenerator.FieldNodeInfo node = - (ChunkInputStreamGenerator.FieldNodeInfo) noChunk.asObjectChunk().get(i); + final ChunkWriter.FieldNodeInfo node = + (ChunkWriter.FieldNodeInfo) noChunk.asObjectChunk().get(i); FieldNode.createFieldNode(header, node.numElements, node.nullCount); } nodesOffset = header.endVector(); @@ -833,39 +807,39 @@ private void processBatches(Consumer visitor, final RecordBa batchSize = Math.min(maxBatchSize, Math.max(1, (int) ((double) rowLimit * 0.9))); } } catch (SizeException ex) { - // was an overflow in the ChunkInputStream generator (probably VarBinary). We can't compute the + // was an overflow in the ChunkInputStream writer (probably VarBinary). We can't compute the // correct number of rows from this failure, so cut batch size in half and try again. This may // occur multiple times until the size is restricted properly if (batchSize == 1) { // this row exceeds internal limits and can never be sent throw (new UncheckedDeephavenException( - "BarrageStreamGenerator - single row (" + offset + ") exceeds transmissible size", ex)); + "BarrageStreamWriterImpl - single row (" + offset + ") exceeds transmissible size", ex)); } final int maximumSize = LongSizedDataStructure.intSize( - "BarrageStreamGenerator", ex.getMaximumSize()); + "BarrageStreamWriterImpl", ex.getMaximumSize()); batchSize = maximumSize >= batchSize ? batchSize / 2 : maximumSize; } } } - private static int findGeneratorForOffset(final List generators, final long offset) { + private static int findWriterForOffset(final ChunkWriter.Context[] chunks, final long offset) { // fast path for smaller updates - if (generators.size() <= 1) { + if (chunks.length <= 1) { return 0; } int low = 0; - int high = generators.size(); + int high = chunks.length; while (low + 1 < high) { int mid = (low + high) / 2; - int cmp = Long.compare(generators.get(mid).getRowOffset(), offset); + int cmp = Long.compare(chunks[mid].getRowOffset(), offset); if (cmp < 0) { - // the generator's first key is low enough + // the writer's first key is low enough low = mid; } else if (cmp > 0) { - // the generator's first key is too high + // the writer's first key is too high high = mid; } else { // first key matches @@ -873,21 +847,21 @@ private static int findGeneratorForOffset(final List } } - // desired generator is at low as the high is exclusive + // desired writer is at low as the high is exclusive return low; } private int appendAddColumns(final RecordBatchMessageView view, final long startRange, final int targetBatchSize, final Consumer addStream, - final ChunkInputStreamGenerator.FieldNodeListener fieldNodeListener, - final ChunkInputStreamGenerator.BufferListener bufferListener) throws IOException { + final ChunkWriter.FieldNodeListener fieldNodeListener, + final ChunkWriter.BufferListener bufferListener) throws IOException { if (addColumnData.length == 0) { return view.addRowOffsets().intSize(); } - // find the generator for the initial position-space key + // find the writer for the initial position-space key long startPos = view.addRowOffsets().get(startRange); - int chunkIdx = findGeneratorForOffset(addColumnData[0].generators(), startPos); + int chunkIdx = findWriterForOffset(addColumnData[0].chunks(), startPos); // adjust the batch size if we would cross a chunk boundary long shift = 0; @@ -895,45 +869,44 @@ private int appendAddColumns(final RecordBatchMessageView view, final long start if (endPos == RowSet.NULL_ROW_KEY) { endPos = Long.MAX_VALUE; } - if (!addColumnData[0].generators().isEmpty()) { - final ChunkInputStreamGenerator tmpGenerator = addColumnData[0].generators().get(chunkIdx); - endPos = Math.min(endPos, tmpGenerator.getLastRowOffset()); - shift = -tmpGenerator.getRowOffset(); + if (addColumnData[0].chunks().length != 0) { + final ChunkWriter.Context writer = addColumnData[0].chunks()[chunkIdx]; + endPos = Math.min(endPos, writer.getLastRowOffset()); + shift = -writer.getRowOffset(); } - // all column generators have the same boundaries, so we can re-use the offsets internal to this chunkIdx + // all column writers have the same boundaries, so we can re-use the offsets internal to this chunkIdx try (final RowSet allowedRange = RowSetFactory.fromRange(startPos, endPos); final WritableRowSet myAddedOffsets = view.addRowOffsets().intersect(allowedRange); final RowSet adjustedOffsets = shift == 0 ? null : myAddedOffsets.shift(shift)) { // every column must write to the stream - for (final ChunkListInputStreamGenerator data : addColumnData) { - final int numElements = data.generators().isEmpty() + for (final ChunkListWriter> chunkListWriter : addColumnData) { + final int numElements = chunkListWriter.chunks().length == 0 ? 0 - : myAddedOffsets.intSize("BarrageStreamGenerator"); + : myAddedOffsets.intSize("BarrageStreamWriterImpl"); if (view.options().columnsAsList()) { // if we are sending columns as a list, we need to add the list buffers before each column - final SingleElementListHeaderInputStreamGenerator listHeader = - new SingleElementListHeaderInputStreamGenerator(numElements); + final SingleElementListHeaderWriter listHeader = + new SingleElementListHeaderWriter(numElements); listHeader.visitFieldNodes(fieldNodeListener); listHeader.visitBuffers(bufferListener); addStream.accept(listHeader); } if (numElements == 0) { - // use an empty generator to publish the column data - try (final RowSet empty = RowSetFactory.empty()) { - final ChunkInputStreamGenerator.DrainableColumn drainableColumn = - data.empty(view.options(), empty); - drainableColumn.visitFieldNodes(fieldNodeListener); - drainableColumn.visitBuffers(bufferListener); + // use an empty writer to publish the column data + final ChunkWriter.DrainableColumn drainableColumn = chunkListWriter.empty(view.options()); + drainableColumn.visitFieldNodes(fieldNodeListener); + drainableColumn.visitBuffers(bufferListener); - // Add the drainable last as it is allowed to immediately close a row set the visitors need - addStream.accept(drainableColumn); - } + // Add the drainable last as it is allowed to immediately close a row set the visitors need + addStream.accept(drainableColumn); } else { - final ChunkInputStreamGenerator generator = data.generators().get(chunkIdx); - final ChunkInputStreamGenerator.DrainableColumn drainableColumn = - generator.getInputStream(view.options(), shift == 0 ? myAddedOffsets : adjustedOffsets); + final ChunkWriter.Context> chunk = chunkListWriter.chunks()[chunkIdx]; + final ChunkWriter.DrainableColumn drainableColumn = chunkListWriter.writer().getInputStream( + chunk, + shift == 0 ? myAddedOffsets : adjustedOffsets, + view.options()); drainableColumn.visitFieldNodes(fieldNodeListener); drainableColumn.visitBuffers(bufferListener); // Add the drainable last as it is allowed to immediately close a row set the visitors need @@ -946,8 +919,8 @@ private int appendAddColumns(final RecordBatchMessageView view, final long start private int appendModColumns(final RecordBatchMessageView view, final long startRange, final int targetBatchSize, final Consumer addStream, - final ChunkInputStreamGenerator.FieldNodeListener fieldNodeListener, - final ChunkInputStreamGenerator.BufferListener bufferListener) throws IOException { + final ChunkWriter.FieldNodeListener fieldNodeListener, + final ChunkWriter.BufferListener bufferListener) throws IOException { int[] columnChunkIdx = new int[modColumnData.length]; // for each column identify the chunk that holds this startRange @@ -955,9 +928,9 @@ private int appendModColumns(final RecordBatchMessageView view, final long start // adjust the batch size if we would cross a chunk boundary for (int ii = 0; ii < modColumnData.length; ++ii) { - final ModColumnGenerator mcd = modColumnData[ii]; - final List generators = mcd.data.generators(); - if (generators.isEmpty()) { + final ModColumnWriter mcd = modColumnData[ii]; + final ChunkWriter.Context[] chunks = mcd.chunkListWriter.chunks(); + if (chunks.length == 0) { continue; } @@ -965,9 +938,9 @@ private int appendModColumns(final RecordBatchMessageView view, final long start // if all mods are being sent, then offsets yield an identity mapping final long startPos = modOffsets != null ? modOffsets.get(startRange) : startRange; if (startPos != RowSet.NULL_ROW_KEY) { - final int chunkIdx = findGeneratorForOffset(generators, startPos); - if (chunkIdx < generators.size() - 1) { - maxLength = Math.min(maxLength, generators.get(chunkIdx).getLastRowOffset() + 1 - startPos); + final int chunkIdx = findWriterForOffset(chunks, startPos); + if (chunkIdx < chunks.length - 1) { + maxLength = Math.min(maxLength, chunks[chunkIdx].getLastRowOffset() + 1 - startPos); } columnChunkIdx[ii] = chunkIdx; } @@ -976,10 +949,10 @@ private int appendModColumns(final RecordBatchMessageView view, final long start // now add mod-column streams, and write the mod column indexes long numRows = 0; for (int ii = 0; ii < modColumnData.length; ++ii) { - final ModColumnGenerator mcd = modColumnData[ii]; - final ChunkInputStreamGenerator generator = mcd.data.generators().isEmpty() + final ModColumnWriter mcd = modColumnData[ii]; + final ChunkWriter.Context> chunk = mcd.chunkListWriter.chunks().length == 0 ? null - : mcd.data.generators().get(columnChunkIdx[ii]); + : mcd.chunkListWriter.chunks()[columnChunkIdx[ii]]; final RowSet modOffsets = view.modRowOffsets(ii); long startPos, endPos; @@ -994,8 +967,8 @@ private int appendModColumns(final RecordBatchMessageView view, final long start // if all mods are being sent, then offsets yield an identity mapping startPos = startRange; endPos = startRange + maxLength - 1; - if (generator != null) { - endPos = Math.min(endPos, generator.getLastRowOffset()); + if (chunk != null) { + endPos = Math.min(endPos, chunk.getLastRowOffset()); } } @@ -1013,32 +986,30 @@ private int appendModColumns(final RecordBatchMessageView view, final long start numRows = Math.max(numRows, myModOffsets.size()); try { - final int numElements = generator == null ? 0 : myModOffsets.intSize("BarrageStreamGenerator"); + final int numElements = chunk == null ? 0 : myModOffsets.intSize("BarrageStreamWriterImpl"); if (view.options().columnsAsList()) { // if we are sending columns as a list, we need to add the list buffers before each column - final SingleElementListHeaderInputStreamGenerator listHeader = - new SingleElementListHeaderInputStreamGenerator(numElements); + final SingleElementListHeaderWriter listHeader = + new SingleElementListHeaderWriter(numElements); listHeader.visitFieldNodes(fieldNodeListener); listHeader.visitBuffers(bufferListener); addStream.accept(listHeader); } if (numElements == 0) { - // use the empty generator to publish the column data - try (final RowSet empty = RowSetFactory.empty()) { - final ChunkInputStreamGenerator.DrainableColumn drainableColumn = - mcd.data.empty(view.options(), empty); - drainableColumn.visitFieldNodes(fieldNodeListener); - drainableColumn.visitBuffers(bufferListener); - // Add the drainable last as it is allowed to immediately close a row set the visitors need - addStream.accept(drainableColumn); - } + // use the empty writer to publish the column data + final ChunkWriter.DrainableColumn drainableColumn = + mcd.chunkListWriter.empty(view.options()); + drainableColumn.visitFieldNodes(fieldNodeListener); + drainableColumn.visitBuffers(bufferListener); + // Add the drainable last as it is allowed to immediately close a row set the visitors need + addStream.accept(drainableColumn); } else { - final long shift = -generator.getRowOffset(); + final long shift = -chunk.getRowOffset(); // normalize to the chunk offsets try (final WritableRowSet adjustedOffsets = shift == 0 ? null : myModOffsets.shift(shift)) { - final ChunkInputStreamGenerator.DrainableColumn drainableColumn = - generator.getInputStream(view.options(), shift == 0 ? myModOffsets : adjustedOffsets); + final ChunkWriter.DrainableColumn drainableColumn = mcd.chunkListWriter.writer().getInputStream( + chunk, shift == 0 ? myModOffsets : adjustedOffsets, view.options()); drainableColumn.visitFieldNodes(fieldNodeListener); drainableColumn.visitBuffers(bufferListener); // Add the drainable last as it is allowed to immediately close a row set the visitors need @@ -1052,7 +1023,7 @@ private int appendModColumns(final RecordBatchMessageView view, final long start return Math.toIntExact(numRows); } - public static abstract class ByteArrayGenerator { + public static abstract class ByteArrayWriter { protected int len; protected byte[] raw; @@ -1061,12 +1032,11 @@ protected int addToFlatBuffer(final FlatBufferBuilder builder) { } } - public static class RowSetGenerator extends ByteArrayGenerator implements SafeCloseable { + public static class RowSetWriter extends ByteArrayWriter implements SafeCloseable { private final RowSet original; - public RowSetGenerator(final RowSet rowSet) throws IOException { + public RowSetWriter(final RowSet rowSet) throws IOException { this.original = rowSet.copy(); - // noinspection UnstableApiUsage try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); final LittleEndianDataOutputStream oos = new LittleEndianDataOutputStream(baos)) { ExternalizableRowSetUtils.writeExternalCompressedDeltas(oos, rowSet); @@ -1099,7 +1069,6 @@ protected int addToFlatBuffer(final RowSet viewport, final FlatBufferBuilder bui final int nlen; final byte[] nraw; - // noinspection UnstableApiUsage try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); final LittleEndianDataOutputStream oos = new LittleEndianDataOutputStream(baos); final RowSet viewOfOriginal = original.intersect(viewport)) { @@ -1113,8 +1082,8 @@ protected int addToFlatBuffer(final RowSet viewport, final FlatBufferBuilder bui } } - public static class BitSetGenerator extends ByteArrayGenerator { - public BitSetGenerator(final BitSet bitset) { + public static class BitSetWriter extends ByteArrayWriter { + public BitSetWriter(final BitSet bitset) { BitSet original = bitset == null ? new BitSet() : bitset; this.raw = original.toByteArray(); final int nBits = original.previousSetBit(Integer.MAX_VALUE - 1) + 1; @@ -1122,8 +1091,8 @@ public BitSetGenerator(final BitSet bitset) { } } - public static class RowSetShiftDataGenerator extends ByteArrayGenerator { - public RowSetShiftDataGenerator(final RowSetShiftData shifted) throws IOException { + public static class RowSetShiftDataWriter extends ByteArrayWriter { + public RowSetShiftDataWriter(final RowSetShiftData shifted) throws IOException { final RowSetBuilderSequential sRangeBuilder = RowSetFactory.builderSequential(); final RowSetBuilderSequential eRangeBuilder = RowSetFactory.builderSequential(); final RowSetBuilderSequential destBuilder = RowSetFactory.builderSequential(); @@ -1143,7 +1112,6 @@ public RowSetShiftDataGenerator(final RowSetShiftData shifted) throws IOExceptio } } - // noinspection UnstableApiUsage try (final RowSet sRange = sRangeBuilder.build(); final RowSet eRange = eRangeBuilder.build(); final RowSet dest = destBuilder.build(); @@ -1159,17 +1127,17 @@ public RowSetShiftDataGenerator(final RowSetShiftData shifted) throws IOExceptio } } - private static final class EmptyRowSetGenerator extends RowSetGenerator { - public static final EmptyRowSetGenerator INSTANCE; + private static final class EmptyRowSetWriter extends RowSetWriter { + public static final EmptyRowSetWriter INSTANCE; static { try { - INSTANCE = new EmptyRowSetGenerator(); + INSTANCE = new EmptyRowSetWriter(); } catch (final IOException ioe) { throw new UncheckedDeephavenException(ioe); } } - EmptyRowSetGenerator() throws IOException { + EmptyRowSetWriter() throws IOException { super(RowSetFactory.empty()); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSnapshotOptions.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSnapshotOptions.java index 7993a5f663c..01b16021630 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSnapshotOptions.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSnapshotOptions.java @@ -6,13 +6,13 @@ import com.google.flatbuffers.FlatBufferBuilder; import io.deephaven.annotations.BuildableStyle; import io.deephaven.barrage.flatbuf.BarrageSnapshotRequest; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; +import io.deephaven.extensions.barrage.chunk.ChunkReader; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; @Immutable @BuildableStyle -public abstract class BarrageSnapshotOptions implements StreamReaderOptions { +public abstract class BarrageSnapshotOptions implements ChunkReader.Options { public static Builder builder() { return ImmutableBarrageSnapshotOptions.builder(); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSubscriptionOptions.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSubscriptionOptions.java index 64e5d13219c..e7ef80e591d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSubscriptionOptions.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageSubscriptionOptions.java @@ -6,13 +6,13 @@ import com.google.flatbuffers.FlatBufferBuilder; import io.deephaven.annotations.BuildableStyle; import io.deephaven.barrage.flatbuf.BarrageSubscriptionRequest; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; +import io.deephaven.extensions.barrage.chunk.ChunkReader; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; @Immutable @BuildableStyle -public abstract class BarrageSubscriptionOptions implements StreamReaderOptions { +public abstract class BarrageSubscriptionOptions implements ChunkReader.Options { public static Builder builder() { return ImmutableBarrageSubscriptionOptions.builder(); diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListInputStreamGenerator.java deleted file mode 100644 index a5b95f2c524..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListInputStreamGenerator.java +++ /dev/null @@ -1,56 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage; - -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.SafeCloseable; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; - -public class ChunkListInputStreamGenerator implements SafeCloseable { - private final List generators; - private final ChunkInputStreamGenerator emptyGenerator; - - public ChunkListInputStreamGenerator(ChunkInputStreamGenerator.Factory factory, Class type, - Class componentType, List> data, - ChunkType chunkType) { - // create an input stream generator for each chunk - ChunkInputStreamGenerator[] generators = new ChunkInputStreamGenerator[data.size()]; - - long rowOffset = 0; - for (int i = 0; i < data.size(); ++i) { - final Chunk valuesChunk = data.get(i); - generators[i] = factory.makeInputStreamGenerator(chunkType, type, componentType, - valuesChunk, rowOffset); - rowOffset += valuesChunk.size(); - } - this.generators = Arrays.asList(generators); - emptyGenerator = factory.makeInputStreamGenerator( - chunkType, type, componentType, chunkType.getEmptyChunk(), 0); - } - - public List generators() { - return generators; - } - - public ChunkInputStreamGenerator.DrainableColumn empty(StreamReaderOptions options, RowSet rowSet) - throws IOException { - return emptyGenerator.getInputStream(options, rowSet); - } - - @Override - public void close() { - for (ChunkInputStreamGenerator generator : generators) { - generator.close(); - } - emptyGenerator.close(); - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListWriter.java new file mode 100644 index 00000000000..d579f28b6a1 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListWriter.java @@ -0,0 +1,54 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.extensions.barrage.chunk.ChunkReader; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; + +import java.io.IOException; +import java.util.List; + +public class ChunkListWriter> implements SafeCloseable { + private final ChunkWriter writer; + private final ChunkWriter.Context[] contexts; + + public ChunkListWriter( + final ChunkWriter writer, + final List chunks) { + this.writer = writer; + + // noinspection unchecked + this.contexts = (ChunkWriter.Context[]) new ChunkWriter.Context[chunks.size()]; + + long rowOffset = 0; + for (int i = 0; i < chunks.size(); ++i) { + final SourceChunkType valuesChunk = chunks.get(i); + this.contexts[i] = writer.makeContext(valuesChunk, rowOffset); + rowOffset += valuesChunk.size(); + } + } + + public ChunkWriter writer() { + return writer; + } + + public ChunkWriter.Context[] chunks() { + return contexts; + } + + public ChunkWriter.DrainableColumn empty(@NotNull final ChunkReader.Options options) throws IOException { + return writer.getEmptyInputStream(options); + } + + @Override + public void close() { + for (final ChunkWriter.Context context : contexts) { + context.decrementReferenceCount(); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkInputStreamGenerator.java deleted file mode 100644 index f51da87e959..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkInputStreamGenerator.java +++ /dev/null @@ -1,134 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.util.referencecounting.ReferenceCounted; - -import java.io.IOException; - -public abstract class BaseChunkInputStreamGenerator> - extends ReferenceCounted - implements ChunkInputStreamGenerator { - - public static final byte[] PADDING_BUFFER = new byte[8]; - public static final int REMAINDER_MOD_8_MASK = 0x7; - - protected final T chunk; - protected final int elementSize; - - private final long rowOffset; - - BaseChunkInputStreamGenerator(final T chunk, final int elementSize, final long rowOffset) { - super(1); - this.chunk = chunk; - this.elementSize = elementSize; - this.rowOffset = rowOffset; - } - - @Override - public long getRowOffset() { - return rowOffset; - } - - @Override - public long getLastRowOffset() { - return rowOffset + chunk.size() - 1; - } - - @Override - public void close() { - decrementReferenceCount(); - } - - @Override - protected void onReferenceCountAtZero() { - if (chunk instanceof PoolableChunk) { - ((PoolableChunk) chunk).close(); - } - } - - /** - * Returns expected size of validity map in bytes. - * - * @param numElements the number of rows - * @return number of bytes to represent the validity buffer for numElements - */ - protected static int getValidityMapSerializationSizeFor(final int numElements) { - return getNumLongsForBitPackOfSize(numElements) * 8; - } - - /** - * Returns the number of longs needed to represent a single bit per element. - * - * @param numElements the number of rows - * @return number of longs needed to represent numElements bits rounded up to the nearest long - */ - protected static int getNumLongsForBitPackOfSize(final int numElements) { - return ((numElements + 63) / 64); - } - - abstract class BaseChunkInputStream extends DrainableColumn { - protected final StreamReaderOptions options; - protected final RowSequence subset; - protected boolean read = false; - - BaseChunkInputStream(final T chunk, final StreamReaderOptions options, final RowSet subset) { - this.options = options; - this.subset = chunk.size() == 0 ? RowSequenceFactory.EMPTY - : subset != null ? subset.copy() : RowSequenceFactory.forRange(0, chunk.size() - 1); - BaseChunkInputStreamGenerator.this.incrementReferenceCount(); - // ignore the empty chunk as these are intentionally empty generators that should work for any subset - if (chunk.size() > 0 && this.subset.lastRowKey() >= chunk.size()) { - throw new IllegalStateException( - "Subset " + this.subset + " is out of bounds for chunk of size " + chunk.size()); - } - } - - @Override - public void close() throws IOException { - BaseChunkInputStreamGenerator.this.decrementReferenceCount(); - subset.close(); - } - - protected int getRawSize() throws IOException { - long size = 0; - if (sendValidityBuffer()) { - size += getValidityMapSerializationSizeFor(subset.intSize()); - } - size += elementSize * subset.size(); - return LongSizedDataStructure.intSize("BaseChunkInputStream.getRawSize", size); - } - - @Override - public int available() throws IOException { - final int rawSize = getRawSize(); - final int rawMod8 = rawSize & REMAINDER_MOD_8_MASK; - return (read ? 0 : rawSize + (rawMod8 > 0 ? 8 - rawMod8 : 0)); - } - - /** - * There are two cases we don't send a validity buffer: - the simplest case is following the arrow flight spec, - * which says that if there are no nulls present, the buffer is optional. - Our implementation of nullCount() - * for primitive types will return zero if the useDeephavenNulls flag is set, so the buffer will also be omitted - * in that case. The client's marshaller does not need to be aware of deephaven nulls but in this mode we assume - * the consumer understands which value is the assigned NULL. - */ - protected boolean sendValidityBuffer() { - return nullCount() != 0; - } - } - - protected static final class SerContext { - long accumulator = 0; - long count = 0; - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkReader.java new file mode 100644 index 00000000000..3391cf72340 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkReader.java @@ -0,0 +1,38 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; + +import java.util.function.Function; +import java.util.function.IntFunction; + +public abstract class BaseChunkReader> + implements ChunkReader { + + protected static > T castOrCreateChunk( + final WritableChunk outChunk, + final int numRows, + final IntFunction chunkFactory, + final Function, T> castFunction) { + if (outChunk != null) { + return castFunction.apply(outChunk); + } + final T newChunk = chunkFactory.apply(numRows); + newChunk.setSize(numRows); + return newChunk; + } + + public static ChunkType getChunkTypeFor(final Class dest) { + if (dest == boolean.class || dest == Boolean.class) { + // Note: Internally booleans are passed around as bytes, but the wire format is packed bits. + return ChunkType.Byte; + } else if (dest != null && !dest.isPrimitive()) { + return ChunkType.Object; + } + return ChunkType.fromElementType(dest); + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkWriter.java new file mode 100644 index 00000000000..d67c6df22a5 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BaseChunkWriter.java @@ -0,0 +1,213 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSequenceFactory; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataOutput; +import java.io.IOException; +import java.util.function.Supplier; + +public abstract class BaseChunkWriter> implements ChunkWriter { + + public static final byte[] PADDING_BUFFER = new byte[8]; + public static final int REMAINDER_MOD_8_MASK = 0x7; + + protected final Supplier emptyChunkSupplier; + protected final int elementSize; + protected final boolean dhNullable; + + BaseChunkWriter( + final Supplier emptyChunkSupplier, + final int elementSize, + final boolean dhNullable) { + this.emptyChunkSupplier = emptyChunkSupplier; + this.elementSize = elementSize; + this.dhNullable = dhNullable; + } + + @Override + public final DrainableColumn getEmptyInputStream(final @NotNull ChunkReader.Options options) throws IOException { + return getInputStream(makeContext(emptyChunkSupplier.get(), 0), null, options); + } + + @Override + public Context makeContext( + @NotNull final SourceChunkType chunk, + final long rowOffset) { + return new Context<>(chunk, rowOffset); + } + + abstract class BaseChunkInputStream> extends DrainableColumn { + protected final ContextType context; + protected final RowSequence subset; + protected final ChunkReader.Options options; + + protected boolean read = false; + private int cachedNullCount = -1; + + BaseChunkInputStream( + @NotNull final ContextType context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + this.context = context; + context.incrementReferenceCount(); + this.options = options; + + this.subset = context.size() == 0 ? RowSequenceFactory.EMPTY + : subset != null + ? subset.copy() + : RowSequenceFactory.forRange(0, context.size() - 1); + + // ignore the empty context as these are intentionally empty writers that should work for any subset + if (context.size() > 0 && this.subset.lastRowKey() >= context.size()) { + throw new IllegalStateException( + "Subset " + this.subset + " is out of bounds for context of size " + context.size()); + } + } + + @Override + public void close() throws IOException { + context.decrementReferenceCount(); + subset.close(); + } + + protected int getRawSize() throws IOException { + long size = 0; + if (sendValidityBuffer()) { + size += getValidityMapSerializationSizeFor(subset.intSize()); + } + size += elementSize * subset.size(); + return LongSizedDataStructure.intSize("BaseChunkInputStream.getRawSize", size); + } + + @Override + public int available() throws IOException { + final int rawSize = getRawSize(); + final int rawMod8 = rawSize & REMAINDER_MOD_8_MASK; + return (read ? 0 : rawSize + (rawMod8 > 0 ? 8 - rawMod8 : 0)); + } + + /** + * There are two cases we don't send a validity buffer: - the simplest case is following the arrow flight spec, + * which says that if there are no nulls present, the buffer is optional. - Our implementation of nullCount() + * for primitive types will return zero if the useDeephavenNulls flag is set, so the buffer will also be omitted + * in that case. The client's marshaller does not need to be aware of deephaven nulls but in this mode we assume + * the consumer understands which value is the assigned NULL. + */ + protected boolean sendValidityBuffer() { + return nullCount() != 0; + } + + @Override + public int nullCount() { + if (dhNullable && options.useDeephavenNulls()) { + return 0; + } + if (cachedNullCount == -1) { + cachedNullCount = 0; + final SourceChunkType chunk = context.getChunk(); + subset.forAllRowKeys(row -> { + if (chunk.isNullAt((int) row)) { + ++cachedNullCount; + } + }); + } + return cachedNullCount; + } + + protected long writeValidityBuffer(final DataOutput dos) { + if (!sendValidityBuffer()) { + return 0; + } + + final SerContext context = new SerContext(); + final Runnable flush = () -> { + try { + dos.writeLong(context.accumulator); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + context.accumulator = 0; + context.count = 0; + }; + subset.forAllRowKeys(row -> { + if (!this.context.getChunk().isNullAt((int) row)) { + context.accumulator |= 1L << context.count; + } + if (++context.count == 64) { + flush.run(); + } + }); + if (context.count > 0) { + flush.run(); + } + + return getValidityMapSerializationSizeFor(subset.intSize()); + } + + /** + * @param bufferSize the size of the buffer to pad + * @return the total size of the buffer after padding + */ + protected long padBufferSize(long bufferSize) { + final long bytesExtended = bufferSize & REMAINDER_MOD_8_MASK; + if (bytesExtended > 0) { + bufferSize += 8 - bytesExtended; + } + return bufferSize; + } + + /** + * Write padding bytes to the output stream to ensure proper alignment. + * + * @param dos the output stream + * @param bytesWritten the number of bytes written so far that need to be padded + * @return the number of bytes extended by the padding + * @throws IOException if an error occurs while writing to the output stream + */ + protected long writePadBuffer(final DataOutput dos, long bytesWritten) throws IOException { + final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; + if (bytesExtended == 0) { + return 0; + } + dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); + return 8 - bytesExtended; + } + } + + /** + * Returns expected size of validity map in bytes. + * + * @param numElements the number of rows + * @return number of bytes to represent the validity buffer for numElements + */ + protected static int getValidityMapSerializationSizeFor(final int numElements) { + return getNumLongsForBitPackOfSize(numElements) * 8; + } + + /** + * Returns the number of longs needed to represent a single bit per element. + * + * @param numElements the number of rows + * @return number of longs needed to represent numElements bits rounded up to the nearest long + */ + protected static int getNumLongsForBitPackOfSize(final int numElements) { + return ((numElements + 63) / 64); + } + + protected static final class SerContext { + long accumulator = 0; + long count = 0; + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkReader.java index 9195db956a4..2c289e4e859 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkReader.java @@ -9,15 +9,17 @@ import io.deephaven.chunk.attributes.Values; import io.deephaven.util.BooleanUtils; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; -import static io.deephaven.extensions.barrage.chunk.BaseChunkInputStreamGenerator.getNumLongsForBitPackOfSize; +import static io.deephaven.extensions.barrage.chunk.BaseChunkWriter.getNumLongsForBitPackOfSize; -public class BooleanChunkReader implements ChunkReader { +public class BooleanChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "BooleanChunkReader"; @FunctionalInterface @@ -38,10 +40,14 @@ public BooleanChunkReader(ByteConversion conversion) { } @Override - public WritableChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableByteChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -97,7 +103,7 @@ public WritableChunk readChunk(Iterator chunk, final int offset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java similarity index 51% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java index b376fde388b..3eafbedb3c8 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java @@ -3,17 +3,13 @@ // package io.deephaven.extensions.barrage.chunk; -import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.BooleanUtils; import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.ByteChunk; -import io.deephaven.chunk.WritableByteChunk; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.io.IOException; @@ -21,51 +17,28 @@ import static io.deephaven.util.QueryConstants.*; -public class BooleanChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "BooleanChunkInputStreamGenerator"; +public class BooleanChunkWriter extends BaseChunkWriter> { + private static final String DEBUG_NAME = "BooleanChunkWriter"; + public static final BooleanChunkWriter INSTANCE = new BooleanChunkWriter(); - public static BooleanChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - // This code path is utilized for arrays / vectors, which cannot be reinterpreted. - WritableByteChunk outChunk = WritableByteChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - final Boolean value = inChunk.get(i); - outChunk.set(i, BooleanUtils.booleanAsByte(value)); - } - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new BooleanChunkInputStreamGenerator(outChunk, rowOffset); - } - - BooleanChunkInputStreamGenerator(final ByteChunk chunk, final long rowOffset) { - // note: element size is zero here to indicate that we cannot use the element size as it is in bytes per row - super(chunk, 0, rowOffset); + public BooleanChunkWriter() { + super(ByteChunk::getEmptyChunk, 0, false); } @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new BooleanChunkInputStream(options, subset); + public DrainableColumn getInputStream( + @NotNull final Context> context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new BooleanChunkInputStream(context, subset, options); } - private class BooleanChunkInputStream extends BaseChunkInputStream { - private BooleanChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_BYTE) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; + private class BooleanChunkInputStream extends BaseChunkInputStream>> { + private BooleanChunkInputStream( + @NotNull final Context> context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); } @Override @@ -93,7 +66,6 @@ public void visitBuffers(final BufferListener listener) { } @Override - @SuppressWarnings("UnstableApiUsage") public int drainTo(final OutputStream outputStream) throws IOException { if (read || subset.isEmpty()) { return 0; @@ -102,7 +74,11 @@ public int drainTo(final OutputStream outputStream) throws IOException { long bytesWritten = 0; read = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer final SerContext context = new SerContext(); final Runnable flush = () -> { try { @@ -115,24 +91,8 @@ public int drainTo(final OutputStream outputStream) throws IOException { context.count = 0; }; - if (sendValidityBuffer()) { - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_BYTE) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)); - } - - // write the included values subset.forAllRowKeys(row -> { - final byte byteValue = chunk.get((int) row); + final byte byteValue = this.context.getChunk().get((int) row); if (byteValue != NULL_BYTE) { context.accumulator |= (byteValue > 0 ? 1L : 0L) << context.count; } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java deleted file mode 100644 index d334e031bed..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java +++ /dev/null @@ -1,161 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkInputStreamGenerator and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.primitive.function.ToByteFunction; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.ByteChunk; -import io.deephaven.chunk.WritableByteChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class ByteChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "ByteChunkInputStreamGenerator"; - - public static ByteChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static ByteChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToByteFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableByteChunk outChunk = WritableByteChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsByte(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new ByteChunkInputStreamGenerator(outChunk, Byte.BYTES, rowOffset); - } - - ByteChunkInputStreamGenerator(final ByteChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new ByteChunkInputStream(options, subset); - } - - private class ByteChunkInputStream extends BaseChunkInputStream { - private ByteChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_BYTE) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_BYTE) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final byte val = chunk.get((int) row); - dos.writeByte(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("ByteChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkReader.java index d9a473df93f..612d0920a2d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkReader.java @@ -13,21 +13,38 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; import java.util.function.Function; -import java.util.function.IntFunction; import static io.deephaven.util.QueryConstants.NULL_BYTE; -public class ByteChunkReader implements ChunkReader { +public class ByteChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "ByteChunkReader"; - private final StreamReaderOptions options; + + @FunctionalInterface + public interface ToByteTransformFunction> { + byte get(WireChunkType wireValues, int wireOffset); + } + + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToByteTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableByteChunk::makeWritableChunk, + WritableChunk::asWritableByteChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); + } + + private final ChunkReader.Options options; private final ByteConversion conversion; @FunctionalInterface @@ -37,16 +54,16 @@ public interface ByteConversion { ByteConversion IDENTITY = (byte a) -> a; } - public ByteChunkReader(StreamReaderOptions options) { + public ByteChunkReader(ChunkReader.Options options) { this(options, ByteConversion.IDENTITY); } - public ByteChunkReader(StreamReaderOptions options, ByteConversion conversion) { + public ByteChunkReader(ChunkReader.Options options, ByteConversion conversion) { this.options = options; this.conversion = conversion; } - public ChunkReader transform(Function transform) { + public ChunkReader> transform(Function transform) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { try (final WritableByteChunk inner = ByteChunkReader.this.readChunk( fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { @@ -73,11 +90,15 @@ public ChunkReader transform(Function transform) { } @Override - public WritableByteChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableByteChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -93,9 +114,6 @@ public WritableByteChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -128,23 +146,10 @@ public WritableByteChunk readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( final ByteConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableByteChunk chunk, final int offset) throws IOException { if (conversion == ByteConversion.IDENTITY) { @@ -163,7 +168,7 @@ private static void useDeephavenNulls( private static void useValidityBuffer( final ByteConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableByteChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkWriter.java new file mode 100644 index 00000000000..fc86adae55c --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkWriter.java @@ -0,0 +1,101 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkWriter and run "./gradlew replicateBarrageUtils" to regenerate +// +// @formatter:off +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.ByteChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class ByteChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "ByteChunkWriter"; + public static final ByteChunkWriter> INSTANCE = new ByteChunkWriter<>( + ByteChunk::getEmptyChunk, ByteChunk::get); + + @FunctionalInterface + public interface ToByteTransformFunction> { + byte get(SourceChunkType sourceValues, int offset); + } + + private final ToByteTransformFunction transform; + + public ByteChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToByteTransformFunction transform) { + super(emptyChunkSupplier, Byte.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new ByteChunkInputStream(context, subset, options); + } + + private class ByteChunkInputStream extends BaseChunkInputStream> { + private ByteChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeByte(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java deleted file mode 100644 index 83b1f2f72f1..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java +++ /dev/null @@ -1,157 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.primitive.function.ToCharFunction; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.CharChunk; -import io.deephaven.chunk.WritableCharChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class CharChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "CharChunkInputStreamGenerator"; - - public static CharChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static CharChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToCharFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableCharChunk outChunk = WritableCharChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsChar(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new CharChunkInputStreamGenerator(outChunk, Character.BYTES, rowOffset); - } - - CharChunkInputStreamGenerator(final CharChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new CharChunkInputStream(options, subset); - } - - private class CharChunkInputStream extends BaseChunkInputStream { - private CharChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_CHAR) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_CHAR) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final char val = chunk.get((int) row); - dos.writeChar(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("CharChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkReader.java index d3fc3ed47a7..96f3984db84 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkReader.java @@ -9,21 +9,38 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; import java.util.function.Function; -import java.util.function.IntFunction; import static io.deephaven.util.QueryConstants.NULL_CHAR; -public class CharChunkReader implements ChunkReader { +public class CharChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "CharChunkReader"; - private final StreamReaderOptions options; + + @FunctionalInterface + public interface ToCharTransformFunction> { + char get(WireChunkType wireValues, int wireOffset); + } + + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToCharTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableCharChunk::makeWritableChunk, + WritableChunk::asWritableCharChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); + } + + private final ChunkReader.Options options; private final CharConversion conversion; @FunctionalInterface @@ -33,16 +50,16 @@ public interface CharConversion { CharConversion IDENTITY = (char a) -> a; } - public CharChunkReader(StreamReaderOptions options) { + public CharChunkReader(ChunkReader.Options options) { this(options, CharConversion.IDENTITY); } - public CharChunkReader(StreamReaderOptions options, CharConversion conversion) { + public CharChunkReader(ChunkReader.Options options, CharConversion conversion) { this.options = options; this.conversion = conversion; } - public ChunkReader transform(Function transform) { + public ChunkReader> transform(Function transform) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { try (final WritableCharChunk inner = CharChunkReader.this.readChunk( fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { @@ -69,11 +86,15 @@ public ChunkReader transform(Function transform) { } @Override - public WritableCharChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableCharChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -89,9 +110,6 @@ public WritableCharChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -124,23 +142,10 @@ public WritableCharChunk readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( final CharConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableCharChunk chunk, final int offset) throws IOException { if (conversion == CharConversion.IDENTITY) { @@ -159,7 +164,7 @@ private static void useDeephavenNulls( private static void useValidityBuffer( final CharConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableCharChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkWriter.java new file mode 100644 index 00000000000..e3875c635b3 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkWriter.java @@ -0,0 +1,97 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.CharChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class CharChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "CharChunkWriter"; + public static final CharChunkWriter> INSTANCE = new CharChunkWriter<>( + CharChunk::getEmptyChunk, CharChunk::get); + + @FunctionalInterface + public interface ToCharTransformFunction> { + char get(SourceChunkType sourceValues, int offset); + } + + private final ToCharTransformFunction transform; + + public CharChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToCharTransformFunction transform) { + super(emptyChunkSupplier, Character.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new CharChunkInputStream(context, subset, options); + } + + private class CharChunkInputStream extends BaseChunkInputStream> { + private CharChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeChar(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java deleted file mode 100644 index bfd22d342e4..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java +++ /dev/null @@ -1,117 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.extensions.barrage.util.DefensiveDrainable; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.QueryConstants; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.ChunkType; -import io.deephaven.util.SafeCloseable; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; - -public interface ChunkInputStreamGenerator extends SafeCloseable { - long MS_PER_DAY = 24 * 60 * 60 * 1000L; - long MIN_LOCAL_DATE_VALUE = QueryConstants.MIN_LONG / MS_PER_DAY; - long MAX_LOCAL_DATE_VALUE = QueryConstants.MAX_LONG / MS_PER_DAY; - - /** - * Creator of {@link ChunkInputStreamGenerator} instances. - *

- * This API may not be stable, while the JS API's usages of it are implemented. - */ - interface Factory { - /** - * Returns an instance capable of writing the given chunk - * - * @param chunkType the type of the chunk to be written - * @param type the Java type of the column being written - * @param componentType the Java type of data in an array/vector, or null if irrelevant - * @param chunk the chunk that will be written out to an input stream - * @param rowOffset the offset into the chunk to start writing from - * @return an instance capable of serializing the given chunk - * @param the type of data in the column - */ - ChunkInputStreamGenerator makeInputStreamGenerator( - final ChunkType chunkType, - final Class type, - final Class componentType, - final Chunk chunk, - final long rowOffset); - } - - /** - * Returns the number of rows that were sent before the first row in this generator. - */ - long getRowOffset(); - - /** - * Returns the offset of the final row this generator can produce. - */ - long getLastRowOffset(); - - /** - * Get an input stream optionally position-space filtered using the provided RowSet. - * - * @param options the serializable options for this subscription - * @param subset if provided, is a position-space filter of source data - * @return a single-use DrainableColumn ready to be drained via grpc - */ - DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) throws IOException; - - final class FieldNodeInfo { - public final int numElements; - public final int nullCount; - - public FieldNodeInfo(final int numElements, final int nullCount) { - this.numElements = numElements; - this.nullCount = nullCount; - } - - public FieldNodeInfo(final org.apache.arrow.flatbuf.FieldNode node) { - this(LongSizedDataStructure.intSize("FieldNodeInfo", node.length()), - LongSizedDataStructure.intSize("FieldNodeInfo", node.nullCount())); - } - } - - @FunctionalInterface - interface FieldNodeListener { - void noteLogicalFieldNode(final int numElements, final int nullCount); - } - - @FunctionalInterface - interface BufferListener { - void noteLogicalBuffer(final long length); - } - - abstract class DrainableColumn extends DefensiveDrainable { - /** - * Append the field nde to the flatbuffer payload via the supplied listener. - * - * @param listener the listener to notify for each logical field node in this payload - */ - public abstract void visitFieldNodes(final FieldNodeListener listener); - - /** - * Append the buffer boundaries to the flatbuffer payload via the supplied listener. - * - * @param listener the listener to notify for each sub-buffer in this payload - */ - public abstract void visitBuffers(final BufferListener listener); - - /** - * Count the number of null elements in the outer-most layer of this column (i.e. does not count nested nulls - * inside of arrays) - * - * @return the number of null elements -- 'useDeephavenNulls' counts are always 0 so that we may omit the - * validity buffer - */ - public abstract int nullCount(); - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkReader.java index 09fc51a18cb..571d9ae38f2 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkReader.java @@ -3,12 +3,14 @@ // package io.deephaven.extensions.barrage.chunk; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; +import io.deephaven.extensions.barrage.ColumnConversionMode; +import io.deephaven.util.QueryConstants; +import io.deephaven.util.annotations.FinalDefault; import org.apache.arrow.flatbuf.Field; -import org.apache.arrow.flatbuf.Type; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; @@ -18,7 +20,56 @@ /** * Consumes Flight/Barrage streams and transforms them into WritableChunks. */ -public interface ChunkReader { +public interface ChunkReader> { + interface Options { + /** + * @return whether we encode the validity buffer to express null values or {@link QueryConstants}'s NULL values. + */ + boolean useDeephavenNulls(); + + /** + * @return the conversion mode to use for object columns + */ + ColumnConversionMode columnConversionMode(); + + /** + * @return the ideal number of records to send per record batch + */ + int batchSize(); + + /** + * @return the maximum number of bytes that should be sent in a single message. + */ + int maxMessageSize(); + + /** + * Some Flight clients cannot handle modifications that have irregular column counts. These clients request that + * the server wrap all columns in a list to enable each column having a variable length. + * + * @return true if the columns should be wrapped in a list + */ + default boolean columnsAsList() { + return false; + } + } + + /** + * Reads the given DataInput to extract the next Arrow buffer as a Deephaven Chunk. + * + * @param fieldNodeIter iterator to read fields from the stream + * @param bufferInfoIter iterator to read buffers from the stream + * @param is input stream containing buffers to be read + * @return a Chunk containing the data from the stream + * @throws IOException if an error occurred while reading the stream + */ + @FinalDefault + default ReadChunkType readChunk( + @NotNull Iterator fieldNodeIter, + @NotNull PrimitiveIterator.OfLong bufferInfoIter, + @NotNull DataInput is) throws IOException { + return readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0); + } + /** * Reads the given DataInput to extract the next Arrow buffer as a Deephaven Chunk. * @@ -31,66 +82,55 @@ public interface ChunkReader { * @return a Chunk containing the data from the stream * @throws IOException if an error occurred while reading the stream */ - WritableChunk readChunk(final Iterator fieldNodeIter, - final PrimitiveIterator.OfLong bufferInfoIter, - final DataInput is, - final WritableChunk outChunk, - final int outOffset, - final int totalRows) throws IOException; + ReadChunkType readChunk( + @NotNull Iterator fieldNodeIter, + @NotNull PrimitiveIterator.OfLong bufferInfoIter, + @NotNull DataInput is, + @Nullable WritableChunk outChunk, + int outOffset, + int totalRows) throws IOException; /** * Supports creation of {@link ChunkReader} instances to use when processing a flight stream. JVM implementations - * for client and server should probably use {@link DefaultChunkReadingFactory#INSTANCE}. + * for client and server should probably use {@link DefaultChunkReaderFactory#INSTANCE}. */ interface Factory { /** * Returns a {@link ChunkReader} for the specified arguments. * - * @param options options for reading the stream - * @param factor a multiplicative factor to apply when reading integers * @param typeInfo the type of data to read into a chunk - * @return a ChunkReader based on the given options, factory, and type to read - */ - ChunkReader getReader(final StreamReaderOptions options, final int factor, final TypeInfo typeInfo); - - /** - * Returns a {@link ChunkReader} for the specified arguments. - * * @param options options for reading the stream - * @param typeInfo the type of data to read into a chunk * @return a ChunkReader based on the given options, factory, and type to read */ - default ChunkReader getReader(final StreamReaderOptions options, final TypeInfo typeInfo) { - return getReader(options, 1, typeInfo); - } - + > ChunkReader newReader( + @NotNull TypeInfo typeInfo, + @NotNull Options options); } /** * Describes type info used by factory implementations when creating a ChunkReader. */ class TypeInfo { - private final ChunkType chunkType; private final Class type; + @Nullable private final Class componentType; private final Field arrowField; - public TypeInfo(ChunkType chunkType, Class type, Class componentType, Field arrowField) { - this.chunkType = chunkType; + public TypeInfo( + @NotNull final Class type, + @Nullable final Class componentType, + @NotNull final Field arrowField) { this.type = type; this.componentType = componentType; this.arrowField = arrowField; } - public ChunkType chunkType() { - return chunkType; - } - public Class type() { return type; } + @Nullable public Class componentType() { return componentType; } @@ -98,28 +138,20 @@ public Class componentType() { public Field arrowField() { return arrowField; } - - public Field componentArrowField() { - if (arrowField.typeType() != Type.List) { - throw new IllegalStateException("Not a flight List"); - } - if (arrowField.childrenLength() != 1) { - throw new IllegalStateException("Incorrect number of child Fields"); - } - return arrowField.children(0); - } } /** * Factory method to create a TypeInfo instance. * - * @param chunkType the output chunk type * @param type the Java type to be read into the chunk * @param componentType the Java type of nested components * @param arrowField the Arrow type to be read into the chunk * @return a TypeInfo instance */ - static TypeInfo typeInfo(ChunkType chunkType, Class type, Class componentType, Field arrowField) { - return new TypeInfo(chunkType, type, componentType, arrowField); + static TypeInfo typeInfo( + @NotNull final Class type, + @Nullable final Class componentType, + @NotNull final Field arrowField) { + return new TypeInfo(type, componentType, arrowField); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkWriter.java new file mode 100644 index 00000000000..8af6c3281c2 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkWriter.java @@ -0,0 +1,171 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.util.pools.PoolableChunk; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.extensions.barrage.util.DefensiveDrainable; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.Chunk; +import io.deephaven.util.referencecounting.ReferenceCounted; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; + +public interface ChunkWriter> { + long MS_PER_DAY = 24 * 60 * 60 * 1000L; + long NS_PER_MS = 1_000_000L; + long NS_PER_DAY = MS_PER_DAY * NS_PER_MS; + + /** + * Creator of {@link ChunkWriter} instances. + *

+ * This API may not be stable, while the JS API's usages of it are implemented. + */ + interface Factory { + /** + * Returns a {@link ChunkReader} for the specified arguments. + * + * @param typeInfo the type of data to read into a chunk + * @return a ChunkReader based on the given options, factory, and type to read + */ + > ChunkWriter newWriter( + @NotNull ChunkReader.TypeInfo typeInfo); + } + + /** + * Create a context for the given chunk. + * + * @param chunk the chunk of data to be written + * @param rowOffset the number of rows that were sent before the first row in this logical message + * @return a context for the given chunk + */ + Context makeContext(final SourceChunkType chunk, final long rowOffset); + + /** + * Get an input stream optionally position-space filtered using the provided RowSet. + * + * @param context the chunk writer context holding the data to be drained to the client + * @param subset if provided, is a position-space filter of source data + * @param options options for reading the stream + * @return a single-use DrainableColumn ready to be drained via grpc + */ + DrainableColumn getInputStream( + @NotNull Context context, + @Nullable RowSet subset, + @NotNull ChunkReader.Options options) throws IOException; + + /** + * Get an input stream representing the empty wire payload for this writer. + * + * @param options options for reading the stream + * @return a single-use DrainableColumn ready to be drained via grpc + */ + DrainableColumn getEmptyInputStream( + @NotNull ChunkReader.Options options) throws IOException; + + class Context> extends ReferenceCounted implements SafeCloseable { + private final T chunk; + private final long rowOffset; + + public Context(final T chunk, final long rowOffset) { + super(1); + this.chunk = chunk; + this.rowOffset = rowOffset; + } + + /** + * @return the chunk wrapped by this wrapper + */ + T getChunk() { + return chunk; + } + + /** + * @return the number of rows that were sent before the first row in this writer. + */ + public long getRowOffset() { + return rowOffset; + } + + /** + * @return the offset of the final row this writer can produce. + */ + public long getLastRowOffset() { + return rowOffset + chunk.size() - 1; + } + + /** + * @return the number of rows in the wrapped chunk + */ + public int size() { + return chunk.size(); + } + + @Override + public void close() { + decrementReferenceCount(); + } + + @Override + protected void onReferenceCountAtZero() { + if (chunk instanceof PoolableChunk) { + ((PoolableChunk) chunk).close(); + } + } + } + + final class FieldNodeInfo { + public final int numElements; + public final int nullCount; + + public FieldNodeInfo(final int numElements, final int nullCount) { + this.numElements = numElements; + this.nullCount = nullCount; + } + + public FieldNodeInfo(final org.apache.arrow.flatbuf.FieldNode node) { + this(LongSizedDataStructure.intSize("FieldNodeInfo", node.length()), + LongSizedDataStructure.intSize("FieldNodeInfo", node.nullCount())); + } + } + + @FunctionalInterface + interface FieldNodeListener { + void noteLogicalFieldNode(final int numElements, final int nullCount); + } + + @FunctionalInterface + interface BufferListener { + void noteLogicalBuffer(final long length); + } + + abstract class DrainableColumn extends DefensiveDrainable { + /** + * Append the field nde to the flatbuffer payload via the supplied listener. + * + * @param listener the listener to notify for each logical field node in this payload + */ + public abstract void visitFieldNodes(final FieldNodeListener listener); + + /** + * Append the buffer boundaries to the flatbuffer payload via the supplied listener. + * + * @param listener the listener to notify for each sub-buffer in this payload + */ + public abstract void visitBuffers(final BufferListener listener); + + /** + * Count the number of null elements in the outer-most layer of this column (i.e. does not count nested nulls + * inside of arrays) + * + * @return the number of null elements -- 'useDeephavenNulls' counts are always 0 so that we may omit the + * validity buffer + */ + public abstract int nullCount(); + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkInputStreamGeneratorFactory.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkInputStreamGeneratorFactory.java deleted file mode 100644 index 8255b870fc1..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkInputStreamGeneratorFactory.java +++ /dev/null @@ -1,178 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import com.google.common.base.Charsets; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.time.DateTimeUtils; -import io.deephaven.util.QueryConstants; -import io.deephaven.vector.Vector; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalTime; -import java.time.ZonedDateTime; - -import static io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.MAX_LOCAL_DATE_VALUE; -import static io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.MIN_LOCAL_DATE_VALUE; -import static io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.MS_PER_DAY; - -/** - * JVM implementation of ChunkInputStreamGenerator.Factory, suitable for use in Java clients and servers. - */ -public class DefaultChunkInputStreamGeneratorFactory implements ChunkInputStreamGenerator.Factory { - public static final DefaultChunkInputStreamGeneratorFactory INSTANCE = - new DefaultChunkInputStreamGeneratorFactory(); - - @Override - public ChunkInputStreamGenerator makeInputStreamGenerator(ChunkType chunkType, Class type, - Class componentType, Chunk chunk, long rowOffset) { - // TODO (deephaven-core#5453): pass in ArrowType to enable ser/deser of single java class in multiple formats - switch (chunkType) { - case Boolean: - throw new UnsupportedOperationException("Booleans are reinterpreted as bytes"); - case Char: - return new CharChunkInputStreamGenerator(chunk.asCharChunk(), Character.BYTES, rowOffset); - case Byte: - if (type == Boolean.class || type == boolean.class) { - // internally we represent booleans as bytes, but the wire format respects arrow's specification - return new BooleanChunkInputStreamGenerator(chunk.asByteChunk(), rowOffset); - } - return new ByteChunkInputStreamGenerator(chunk.asByteChunk(), Byte.BYTES, rowOffset); - case Short: - return new ShortChunkInputStreamGenerator(chunk.asShortChunk(), Short.BYTES, rowOffset); - case Int: - return new IntChunkInputStreamGenerator(chunk.asIntChunk(), Integer.BYTES, rowOffset); - case Long: - return new LongChunkInputStreamGenerator(chunk.asLongChunk(), Long.BYTES, rowOffset); - case Float: - return new FloatChunkInputStreamGenerator(chunk.asFloatChunk(), Float.BYTES, rowOffset); - case Double: - return new DoubleChunkInputStreamGenerator(chunk.asDoubleChunk(), Double.BYTES, rowOffset); - case Object: - if (type.isArray()) { - if (componentType == byte.class) { - return new VarBinaryChunkInputStreamGenerator<>(chunk.asObjectChunk(), rowOffset, - (out, item) -> out.write((byte[]) item)); - } else { - return new VarListChunkInputStreamGenerator<>(this, type, chunk.asObjectChunk(), rowOffset); - } - } - if (Vector.class.isAssignableFrom(type)) { - // noinspection unchecked - return new VectorChunkInputStreamGenerator(this, - (Class>) type, componentType, chunk.asObjectChunk(), rowOffset); - } - if (type == String.class) { - return new VarBinaryChunkInputStreamGenerator(chunk.asObjectChunk(), rowOffset, - (out, str) -> out.write(str.getBytes(Charsets.UTF_8))); - } - if (type == BigInteger.class) { - return new VarBinaryChunkInputStreamGenerator(chunk.asObjectChunk(), rowOffset, - (out, item) -> out.write(item.toByteArray())); - } - if (type == BigDecimal.class) { - return new VarBinaryChunkInputStreamGenerator(chunk.asObjectChunk(), rowOffset, - (out, item) -> { - final BigDecimal normal = item.stripTrailingZeros(); - final int v = normal.scale(); - // Write as little endian, arrow endianness. - out.write(0xFF & v); - out.write(0xFF & (v >> 8)); - out.write(0xFF & (v >> 16)); - out.write(0xFF & (v >> 24)); - out.write(normal.unscaledValue().toByteArray()); - }); - } - if (type == Instant.class) { - // This code path is utilized for arrays and vectors of Instant, which cannot be reinterpreted. - ObjectChunk objChunk = chunk.asObjectChunk(); - WritableLongChunk outChunk = WritableLongChunk.makeWritableChunk(objChunk.size()); - for (int i = 0; i < objChunk.size(); ++i) { - outChunk.set(i, DateTimeUtils.epochNanos(objChunk.get(i))); - } - if (chunk instanceof PoolableChunk) { - ((PoolableChunk) chunk).close(); - } - return new LongChunkInputStreamGenerator(outChunk, Long.BYTES, rowOffset); - } - if (type == ZonedDateTime.class) { - // This code path is utilized for arrays and vectors of Instant, which cannot be reinterpreted. - ObjectChunk objChunk = chunk.asObjectChunk(); - WritableLongChunk outChunk = WritableLongChunk.makeWritableChunk(objChunk.size()); - for (int i = 0; i < objChunk.size(); ++i) { - outChunk.set(i, DateTimeUtils.epochNanos(objChunk.get(i))); - } - if (chunk instanceof PoolableChunk) { - ((PoolableChunk) chunk).close(); - } - return new LongChunkInputStreamGenerator(outChunk, Long.BYTES, rowOffset); - } - if (type == Boolean.class) { - return BooleanChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Byte.class) { - return ByteChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Character.class) { - return CharChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Double.class) { - return DoubleChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Float.class) { - return FloatChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Integer.class) { - return IntChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Long.class) { - return LongChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == Short.class) { - return ShortChunkInputStreamGenerator.convertBoxed(chunk.asObjectChunk(), rowOffset); - } - if (type == LocalDate.class) { - return LongChunkInputStreamGenerator.convertWithTransform(chunk.asObjectChunk(), - rowOffset, date -> { - if (date == null) { - return QueryConstants.NULL_LONG; - } - final long epochDay = date.toEpochDay(); - if (epochDay < MIN_LOCAL_DATE_VALUE || epochDay > MAX_LOCAL_DATE_VALUE) { - throw new IllegalArgumentException("Date out of range: " + date + " (" + epochDay - + " not in [" + MIN_LOCAL_DATE_VALUE + ", " + MAX_LOCAL_DATE_VALUE + "])"); - } - return epochDay * MS_PER_DAY; - }); - } - if (type == LocalTime.class) { - return LongChunkInputStreamGenerator.convertWithTransform(chunk.asObjectChunk(), - rowOffset, time -> { - if (time == null) { - return QueryConstants.NULL_LONG; - } - final long nanoOfDay = time.toNanoOfDay(); - if (nanoOfDay < 0) { - throw new IllegalArgumentException("Time out of range: " + time); - } - return nanoOfDay; - }); - } - // TODO (core#936): support column conversion modes - - return new VarBinaryChunkInputStreamGenerator<>(chunk.asObjectChunk(), rowOffset, - (out, item) -> out.write(item.toString().getBytes(Charsets.UTF_8))); - default: - throw new UnsupportedOperationException(); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReaderFactory.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReaderFactory.java new file mode 100644 index 00000000000..b90b915b63b --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReaderFactory.java @@ -0,0 +1,1244 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import com.google.common.base.Charsets; +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.WritableByteChunk; +import io.deephaven.chunk.WritableCharChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableDoubleChunk; +import io.deephaven.chunk.WritableFloatChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.WritableShortChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.impl.lang.QueryLanguageFunctionUtils; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.extensions.barrage.chunk.array.ArrayExpansionKernel; +import io.deephaven.extensions.barrage.chunk.vector.VectorExpansionKernel; +import io.deephaven.internal.log.LoggerFactory; +import io.deephaven.io.logger.Logger; +import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.QueryConstants; +import io.deephaven.util.type.TypeUtils; +import io.deephaven.vector.Vector; +import org.apache.arrow.vector.PeriodDuration; +import org.apache.arrow.vector.types.TimeUnit; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.jetbrains.annotations.NotNull; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteOrder; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.Period; +import java.time.ZoneId; +import java.time.ZonedDateTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static io.deephaven.extensions.barrage.chunk.ChunkWriter.MS_PER_DAY; + +/** + * JVM implementation of {@link ChunkReader.Factory}, suitable for use in Java clients and servers. This default + * implementation may not round trip flight types in a stable way, but will round trip Deephaven table definitions and + * table data. Neither of these is a required/expected property of being a Flight/Barrage/Deephaven client. + */ +public class DefaultChunkReaderFactory implements ChunkReader.Factory { + static final boolean LITTLE_ENDIAN = ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN; + static final Set SPECIAL_TYPES = Set.of( + ArrowType.ArrowTypeID.List, + ArrowType.ArrowTypeID.FixedSizeList, + ArrowType.ArrowTypeID.Map, + ArrowType.ArrowTypeID.Struct, + ArrowType.ArrowTypeID.Union, + ArrowType.ArrowTypeID.Null); + + public static final Logger log = LoggerFactory.getLogger(DefaultChunkReaderFactory.class); + public static final ChunkReader.Factory INSTANCE = new DefaultChunkReaderFactory(); + + protected interface ChunkReaderFactory { + ChunkReader> make( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options); + } + + // allow subclasses to modify this as they wish + protected final Map, ChunkReaderFactory>> registeredFactories = + new HashMap<>(); + + protected DefaultChunkReaderFactory() { + register(ArrowType.ArrowTypeID.Timestamp, long.class, DefaultChunkReaderFactory::timestampToLong); + register(ArrowType.ArrowTypeID.Timestamp, Instant.class, DefaultChunkReaderFactory::timestampToInstant); + register(ArrowType.ArrowTypeID.Timestamp, ZonedDateTime.class, + DefaultChunkReaderFactory::timestampToZonedDateTime); + register(ArrowType.ArrowTypeID.Timestamp, LocalDateTime.class, + DefaultChunkReaderFactory::timestampToLocalDateTime); + register(ArrowType.ArrowTypeID.Utf8, String.class, DefaultChunkReaderFactory::utf8ToString); + register(ArrowType.ArrowTypeID.Duration, long.class, DefaultChunkReaderFactory::durationToLong); + register(ArrowType.ArrowTypeID.Duration, Duration.class, DefaultChunkReaderFactory::durationToDuration); + register(ArrowType.ArrowTypeID.FloatingPoint, float.class, DefaultChunkReaderFactory::floatingPointToFloat); + register(ArrowType.ArrowTypeID.FloatingPoint, double.class, DefaultChunkReaderFactory::floatingPointToDouble); + register(ArrowType.ArrowTypeID.FloatingPoint, BigDecimal.class, + DefaultChunkReaderFactory::floatingPointToBigDecimal); + register(ArrowType.ArrowTypeID.Binary, byte[].class, DefaultChunkReaderFactory::binaryToByteArray); + register(ArrowType.ArrowTypeID.Binary, BigInteger.class, DefaultChunkReaderFactory::binaryToBigInt); + register(ArrowType.ArrowTypeID.Binary, BigDecimal.class, DefaultChunkReaderFactory::binaryToBigDecimal); + register(ArrowType.ArrowTypeID.Time, long.class, DefaultChunkReaderFactory::timeToLong); + register(ArrowType.ArrowTypeID.Time, LocalTime.class, DefaultChunkReaderFactory::timeToLocalTime); + register(ArrowType.ArrowTypeID.Decimal, byte.class, DefaultChunkReaderFactory::decimalToByte); + register(ArrowType.ArrowTypeID.Decimal, char.class, DefaultChunkReaderFactory::decimalToChar); + register(ArrowType.ArrowTypeID.Decimal, short.class, DefaultChunkReaderFactory::decimalToShort); + register(ArrowType.ArrowTypeID.Decimal, int.class, DefaultChunkReaderFactory::decimalToInt); + register(ArrowType.ArrowTypeID.Decimal, long.class, DefaultChunkReaderFactory::decimalToLong); + register(ArrowType.ArrowTypeID.Decimal, BigInteger.class, DefaultChunkReaderFactory::decimalToBigInteger); + register(ArrowType.ArrowTypeID.Decimal, float.class, DefaultChunkReaderFactory::decimalToFloat); + register(ArrowType.ArrowTypeID.Decimal, double.class, DefaultChunkReaderFactory::decimalToDouble); + register(ArrowType.ArrowTypeID.Decimal, BigDecimal.class, DefaultChunkReaderFactory::decimalToBigDecimal); + register(ArrowType.ArrowTypeID.Int, byte.class, DefaultChunkReaderFactory::intToByte); + register(ArrowType.ArrowTypeID.Int, char.class, DefaultChunkReaderFactory::intToChar); + register(ArrowType.ArrowTypeID.Int, short.class, DefaultChunkReaderFactory::intToShort); + register(ArrowType.ArrowTypeID.Int, int.class, DefaultChunkReaderFactory::intToInt); + register(ArrowType.ArrowTypeID.Int, long.class, DefaultChunkReaderFactory::intToLong); + register(ArrowType.ArrowTypeID.Int, BigInteger.class, DefaultChunkReaderFactory::intToBigInt); + register(ArrowType.ArrowTypeID.Int, float.class, DefaultChunkReaderFactory::intToFloat); + register(ArrowType.ArrowTypeID.Int, double.class, DefaultChunkReaderFactory::intToDouble); + register(ArrowType.ArrowTypeID.Int, BigDecimal.class, DefaultChunkReaderFactory::intToBigDecimal); + register(ArrowType.ArrowTypeID.Bool, boolean.class, DefaultChunkReaderFactory::boolToBoolean); + register(ArrowType.ArrowTypeID.Bool, Boolean.class, DefaultChunkReaderFactory::boolToBoolean); + register(ArrowType.ArrowTypeID.Bool, byte.class, DefaultChunkReaderFactory::boolToBoolean); + register(ArrowType.ArrowTypeID.FixedSizeBinary, byte[].class, + DefaultChunkReaderFactory::fixedSizeBinaryToByteArray); + register(ArrowType.ArrowTypeID.Date, int.class, DefaultChunkReaderFactory::dateToInt); + register(ArrowType.ArrowTypeID.Date, long.class, DefaultChunkReaderFactory::dateToLong); + register(ArrowType.ArrowTypeID.Date, LocalDate.class, DefaultChunkReaderFactory::dateToLocalDate); + register(ArrowType.ArrowTypeID.Interval, long.class, DefaultChunkReaderFactory::intervalToDurationLong); + register(ArrowType.ArrowTypeID.Interval, Duration.class, DefaultChunkReaderFactory::intervalToDuration); + register(ArrowType.ArrowTypeID.Interval, Period.class, DefaultChunkReaderFactory::intervalToPeriod); + register(ArrowType.ArrowTypeID.Interval, PeriodDuration.class, + DefaultChunkReaderFactory::intervalToPeriodDuration); + // TODO NATE NOCOMMIT: Test each of Arrow's timezone formats in Instant -> ZonedDateTime + } + + @Override + public > ChunkReader newReader( + @NotNull final ChunkReader.TypeInfo typeInfo, + @NotNull final ChunkReader.Options options) { + // TODO (deephaven/deephaven-core#6033): Run-End Support + // TODO (deephaven/deephaven-core#6034): Dictionary Support + + final Field field = Field.convertField(typeInfo.arrowField()); + + final ArrowType.ArrowTypeID typeId = field.getType().getTypeID(); + final boolean isSpecialType = SPECIAL_TYPES.contains(typeId); + + // TODO (deephaven/deephaven-core#6038): these arrow types require 64-bit offsets + if (typeId == ArrowType.ArrowTypeID.LargeUtf8 + || typeId == ArrowType.ArrowTypeID.LargeBinary + || typeId == ArrowType.ArrowTypeID.LargeList) { + throw new UnsupportedOperationException(String.format( + "No support for 64-bit offsets to map arrow type %s to %s.", + field.getType().toString(), + typeInfo.type().getCanonicalName())); + } + + final Map, ChunkReaderFactory> knownReaders = registeredFactories.get(typeId); + if (knownReaders == null && !isSpecialType) { + throw new UnsupportedOperationException(String.format( + "No known ChunkReader for arrow type %s to %s.", + field.getType().toString(), + typeInfo.type().getCanonicalName())); + } + + final ChunkReaderFactory chunkReaderFactory = knownReaders == null ? null : knownReaders.get(typeInfo.type()); + if (chunkReaderFactory != null) { + // noinspection unchecked + final ChunkReader reader = (ChunkReader) chunkReaderFactory.make(field.getType(), typeInfo, options); + if (reader != null) { + return reader; + } + } else if (!isSpecialType) { + throw new UnsupportedOperationException(String.format( + "No known ChunkReader for arrow type %s to %s. Supported types: %s", + field.getType().toString(), + typeInfo.type().getCanonicalName(), + knownReaders.keySet().stream().map(Object::toString).collect(Collectors.joining(", ")))); + } + + if (typeId == ArrowType.ArrowTypeID.Null) { + return new NullChunkReader<>(typeInfo.type()); + } + + if (typeId == ArrowType.ArrowTypeID.List + || typeId == ArrowType.ArrowTypeID.FixedSizeList) { + + // TODO (deephaven/deephaven-core#5947): Add SPARSE branch for ListView + int fixedSizeLength = 0; + final ListChunkReader.Mode mode; + if (typeId == ArrowType.ArrowTypeID.List) { + mode = ListChunkReader.Mode.DENSE; + } else { + mode = ListChunkReader.Mode.FIXED; + fixedSizeLength = ((ArrowType.FixedSizeList) field.getType()).getListSize(); + } + + final ChunkReader.TypeInfo componentTypeInfo; + final boolean useVectorKernels = Vector.class.isAssignableFrom(typeInfo.type()); + if (useVectorKernels) { + final Class componentType = + VectorExpansionKernel.getComponentType(typeInfo.type(), typeInfo.componentType()); + componentTypeInfo = new ChunkReader.TypeInfo( + componentType, + componentType.getComponentType(), + typeInfo.arrowField().children(0)); + } else if (typeInfo.type().isArray()) { + final Class componentType = typeInfo.componentType(); + // noinspection DataFlowIssue + componentTypeInfo = new ChunkReader.TypeInfo( + componentType, + componentType.getComponentType(), + typeInfo.arrowField().children(0)); + } else { + throw new UnsupportedOperationException(String.format( + "No known ChunkReader for arrow type %s to %s. Expected destination type to be an array.", + field.getType().toString(), + typeInfo.type().getCanonicalName())); + } + + final ChunkType chunkType = ListChunkReader.getChunkTypeFor(componentTypeInfo.type()); + final ExpansionKernel kernel; + if (useVectorKernels) { + kernel = VectorExpansionKernel.makeExpansionKernel(chunkType, componentTypeInfo.type()); + } else { + kernel = ArrayExpansionKernel.makeExpansionKernel(chunkType, componentTypeInfo.type()); + } + final ChunkReader> componentReader = newReader(componentTypeInfo, options); + + // noinspection unchecked + return (ChunkReader) new ListChunkReader<>(mode, fixedSizeLength, kernel, componentReader); + } + + if (typeId == ArrowType.ArrowTypeID.Map) { + // TODO: can user supply collector? + final Field structField = field.getChildren().get(0); + final Field keyField = structField.getChildren().get(0); + final Field valueField = structField.getChildren().get(1); + + // TODO NATE NOCOMMIT: implement + } + + if (typeId == ArrowType.ArrowTypeID.Struct) { + // TODO: expose transformer API of Map> -> T + // TODO: maybe defaults to Map + // TODO NATE NOCOMMIT: implement + } + + if (typeId == ArrowType.ArrowTypeID.Union) { + // TODO: defaults to Object + final ArrowType.Union unionType = (ArrowType.Union) field.getType(); + switch (unionType.getMode()) { + case Sparse: + // TODO NATE NOCOMMIT: implement + break; + case Dense: + // TODO NATE NOCOMMIT: implement + break; + default: + throw new IllegalArgumentException("Unexpected union mode: " + unionType.getMode()); + } + } + + throw new UnsupportedOperationException(String.format( + "No known ChunkReader for arrow type %s to %s. Arrow type supports: %s", + field.getType().toString(), + typeInfo.type().getCanonicalName(), + knownReaders == null ? "none" + : knownReaders.keySet().stream() + .map(Object::toString) + .collect(Collectors.joining(", ")))); + } + + @SuppressWarnings("unchecked") + protected void register( + final ArrowType.ArrowTypeID arrowType, + final Class deephavenType, + final ChunkReaderFactory chunkReaderFactory) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(deephavenType, chunkReaderFactory); + + // if primitive automatically register the boxed version of this mapping, too + if (deephavenType == byte.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Byte.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } else if (deephavenType == short.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Short.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } else if (deephavenType == int.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Integer.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } else if (deephavenType == long.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Long.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } else if (deephavenType == char.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Character.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } else if (deephavenType == float.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Float.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } else if (deephavenType == double.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Double.class, (at, typeInfo, options) -> transformToObject( + (ChunkReader>) chunkReaderFactory.make(at, typeInfo, options), + (chunk, ii) -> TypeUtils.box(chunk.get(ii)))); + } + } + + private static long factorForTimeUnit(final TimeUnit unit) { + switch (unit) { + case NANOSECOND: + return 1; + case MICROSECOND: + return 1000; + case MILLISECOND: + return 1000 * 1000L; + case SECOND: + return 1000 * 1000 * 1000L; + default: + throw new IllegalArgumentException("Unexpected time unit value: " + unit); + } + } + + private static ChunkReader> timestampToLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final long factor = factorForTimeUnit(((ArrowType.Timestamp) arrowType).getUnit()); + return factor == 1 + ? new LongChunkReader(options) + : new LongChunkReader(options, + (long v) -> v == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : (v * factor)); + } + + private static ChunkReader> timestampToInstant( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final long factor = factorForTimeUnit(((ArrowType.Timestamp) arrowType).getUnit()); + return new FixedWidthChunkReader<>(Long.BYTES, true, options, io -> { + final long value = io.readLong(); + if (value == QueryConstants.NULL_LONG) { + return null; + } + return DateTimeUtils.epochNanosToInstant(value * factor); + }); + } + + private static ChunkReader> timestampToZonedDateTime( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Timestamp tsType = (ArrowType.Timestamp) arrowType; + final String timezone = tsType.getTimezone(); + final ZoneId tz = timezone == null ? ZoneId.systemDefault() : DateTimeUtils.parseTimeZone(timezone); + final long factor = factorForTimeUnit(tsType.getUnit()); + return new FixedWidthChunkReader<>(Long.BYTES, true, options, io -> { + final long value = io.readLong(); + if (value == QueryConstants.NULL_LONG) { + return null; + } + return DateTimeUtils.epochNanosToZonedDateTime(value * factor, tz); + }); + } + + private static ChunkReader> timestampToLocalDateTime( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Timestamp tsType = (ArrowType.Timestamp) arrowType; + final ZoneId tz = DateTimeUtils.parseTimeZone(tsType.getTimezone()); + final long factor = factorForTimeUnit(tsType.getUnit()); + return new FixedWidthChunkReader<>(Long.BYTES, true, options, io -> { + final long value = io.readLong(); + if (value == QueryConstants.NULL_LONG) { + return null; + } + // noinspection DataFlowIssue + return DateTimeUtils.epochNanosToZonedDateTime(value * factor, tz).toLocalDateTime(); + }); + } + + private static ChunkReader> utf8ToString( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new VarBinaryChunkReader<>((buf, off, len) -> new String(buf, off, len, Charsets.UTF_8)); + } + + private static ChunkReader> durationToLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final long factor = factorForTimeUnit(((ArrowType.Duration) arrowType).getUnit()); + return factor == 1 + ? new LongChunkReader(options) + : new LongChunkReader(options, + (long v) -> v == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : (v * factor)); + } + + private static ChunkReader> durationToDuration( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final long factor = factorForTimeUnit(((ArrowType.Duration) arrowType).getUnit()); + return transformToObject(new LongChunkReader(options), (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG ? null : Duration.ofNanos(value * factor); + }); + } + + private static ChunkReader> floatingPointToFloat( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new FloatChunkReader(((ArrowType.FloatingPoint) arrowType).getPrecision().getFlatbufID(), options); + } + + private static ChunkReader> floatingPointToDouble( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new DoubleChunkReader(((ArrowType.FloatingPoint) arrowType).getPrecision().getFlatbufID(), options); + } + + private static ChunkReader> floatingPointToBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return transformToObject( + new DoubleChunkReader(((ArrowType.FloatingPoint) arrowType).getPrecision().getFlatbufID(), options), + (chunk, ii) -> { + double value = chunk.get(ii); + return value == QueryConstants.NULL_DOUBLE ? null : BigDecimal.valueOf(value); + }); + } + + private static ChunkReader> binaryToByteArray( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new VarBinaryChunkReader<>((buf, off, len) -> Arrays.copyOfRange(buf, off, off + len)); + } + + private static ChunkReader> binaryToBigInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new VarBinaryChunkReader<>(BigInteger::new); + } + + private static ChunkReader> binaryToBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new VarBinaryChunkReader<>((final byte[] buf, final int offset, final int length) -> { + // read the int scale value as little endian, arrow's endianness. + final byte b1 = buf[offset]; + final byte b2 = buf[offset + 1]; + final byte b3 = buf[offset + 2]; + final byte b4 = buf[offset + 3]; + final int scale = b4 << 24 | (b3 & 0xFF) << 16 | (b2 & 0xFF) << 8 | (b1 & 0xFF); + return new BigDecimal(new BigInteger(buf, offset + 4, length - 4), scale); + }); + } + + private static ChunkReader> timeToLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // See timeToLocalTime's comment for more information on wire format. + final ArrowType.Time timeType = (ArrowType.Time) arrowType; + final int bitWidth = timeType.getBitWidth(); + final long factor = factorForTimeUnit(timeType.getUnit()); + switch (bitWidth) { + case 32: + return LongChunkReader.transformTo(new IntChunkReader(options), (chunk, ii) -> { + long value = QueryLanguageFunctionUtils.longCast(chunk.get(ii)); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value * factor; + }); + + case 64: + return LongChunkReader.transformTo(new LongChunkReader(options), (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value * factor; + }); + + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> timeToLocalTime( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + /* + * Time is either a 32-bit or 64-bit signed integer type representing an elapsed time since midnight, stored in + * either of four units: seconds, milliseconds, microseconds or nanoseconds. + * + * The integer `bitWidth` depends on the `unit` and must be one of the following: + * @formatter:off + * - SECOND and MILLISECOND: 32 bits + * - MICROSECOND and NANOSECOND: 64 bits + * @formatter:on + * + * The allowed values are between 0 (inclusive) and 86400 (=24*60*60) seconds (exclusive), adjusted for the time + * unit (for example, up to 86400000 exclusive for the MILLISECOND unit). This definition doesn't allow for leap + * seconds. Time values from measurements with leap seconds will need to be corrected when ingesting into Arrow + * (for example by replacing the value 86400 with 86399). + */ + + final ArrowType.Time timeType = (ArrowType.Time) arrowType; + final int bitWidth = timeType.getBitWidth(); + final long factor = factorForTimeUnit(timeType.getUnit()); + switch (bitWidth) { + case 32: + return transformToObject(new IntChunkReader(options), (chunk, ii) -> { + int value = chunk.get(ii); + return value == QueryConstants.NULL_INT ? null : LocalTime.ofNanoOfDay(value * factor); + }); + + case 64: + return transformToObject(new LongChunkReader(options), (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG ? null : LocalTime.ofNanoOfDay(value * factor); + }); + + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> decimalToByte( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return ByteChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToChar( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return CharChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.charCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToShort( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return ShortChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return IntChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return LongChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToBigInteger( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // note this mapping is particularly useful if scale == 0 + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + + return new FixedWidthChunkReader<>(byteWidth, false, options, dataInput -> { + final byte[] value = new byte[byteWidth]; + dataInput.readFully(value); + if (LITTLE_ENDIAN) { + // Decimal stored as native endian, need to swap bytes to make BigDecimal if native endian is LE + byte temp; + int stop = byteWidth / 2; + for (int i = 0, j; i < stop; i++) { + temp = value[i]; + j = (byteWidth - 1) - i; + value[i] = value[j]; + value[j] = temp; + } + } + + BigInteger unscaledValue = new BigInteger(value); + return unscaledValue.divide(BigInteger.ONE.pow(scale)); + }); + } + + private static ChunkReader> decimalToFloat( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return FloatChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.floatCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToDouble( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return DoubleChunkReader.transformTo(decimalToBigDecimal(arrowType, typeInfo, options), + (chunk, ii) -> QueryLanguageFunctionUtils.doubleCast(chunk.get(ii))); + } + + private static ChunkReader> decimalToBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + + return new FixedWidthChunkReader<>(byteWidth, false, options, dataInput -> { + final byte[] value = new byte[byteWidth]; + dataInput.readFully(value); + if (LITTLE_ENDIAN) { + // Decimal stored as native endian, need to swap bytes to make BigDecimal if native endian is LE + byte temp; + int stop = byteWidth / 2; + for (int i = 0, j; i < stop; i++) { + temp = value[i]; + j = (byteWidth - 1) - i; + value[i] = value[j]; + value[j] = temp; + } + } + + BigInteger unscaledValue = new BigInteger(value); + return new BigDecimal(unscaledValue, scale); + }); + } + + private static ChunkReader> intToByte( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + // note unsigned mappings to byte will overflow byte; but user has asked for this + return new ByteChunkReader(options); + case 16: + // note shorts may overflow byte; but user has asked for this + return ByteChunkReader.transformTo(new ShortChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 32: + // note ints may overflow byte; but user has asked for this + return ByteChunkReader.transformTo(new IntChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 64: + // note longs may overflow byte; but user has asked for this + return ByteChunkReader.transformTo(new LongChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> intToShort( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean unsigned = !intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return ShortChunkReader.transformTo(new ByteChunkReader(options), + (chunk, ii) -> maskIfOverflow(unsigned, + Byte.BYTES, QueryLanguageFunctionUtils.shortCast(chunk.get(ii)))); + case 16: + // note unsigned mappings to short will overflow short; but user has asked for this + return new ShortChunkReader(options); + case 32: + // note ints may overflow short; but user has asked for this + return ShortChunkReader.transformTo(new IntChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 64: + // note longs may overflow short; but user has asked for this + return ShortChunkReader.transformTo(new LongChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> intToInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean unsigned = !intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return IntChunkReader.transformTo(new ByteChunkReader(options), + (chunk, ii) -> maskIfOverflow(unsigned, Byte.BYTES, + QueryLanguageFunctionUtils.intCast(chunk.get(ii)))); + case 16: + return IntChunkReader.transformTo(new ShortChunkReader(options), (chunk, ii) -> maskIfOverflow(unsigned, + Short.BYTES, QueryLanguageFunctionUtils.intCast(chunk.get(ii)))); + case 32: + // note unsigned int may overflow int; but user has asked for this + return new IntChunkReader(options); + case 64: + // note longs may overflow int; but user has asked for this + return IntChunkReader.transformTo(new LongChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> intToLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean unsigned = !intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return LongChunkReader.transformTo(new ByteChunkReader(options), + (chunk, ii) -> maskIfOverflow(unsigned, Byte.BYTES, + QueryLanguageFunctionUtils.longCast(chunk.get(ii)))); + case 16: + return LongChunkReader.transformTo(new ShortChunkReader(options), + (chunk, ii) -> maskIfOverflow(unsigned, + Short.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii)))); + case 32: + return LongChunkReader.transformTo(new IntChunkReader(options), (chunk, ii) -> maskIfOverflow(unsigned, + Integer.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii)))); + case 64: + // note unsigned long may overflow long; but user has asked for this + return new LongChunkReader(options); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> intToBigInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean unsigned = !intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return transformToObject(new ByteChunkReader(options), (chunk, ii) -> toBigInt(maskIfOverflow( + unsigned, Byte.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii))))); + case 16: + return transformToObject(new ShortChunkReader(options), (chunk, ii) -> toBigInt(maskIfOverflow( + unsigned, Short.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii))))); + case 32: + return transformToObject(new IntChunkReader(options), (chunk, ii) -> toBigInt(maskIfOverflow( + unsigned, Integer.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii))))); + case 64: + return transformToObject(new LongChunkReader(options), + (chunk, ii) -> maskIfOverflow(unsigned, Long.BYTES, toBigInt(chunk.get(ii)))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> intToFloat( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean signed = intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return FloatChunkReader.transformTo(new ByteChunkReader(options), + (chunk, ii) -> floatCast(Byte.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + case 16: + return FloatChunkReader.transformTo(new ShortChunkReader(options), + (chunk, ii) -> floatCast(Short.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + case 32: + return FloatChunkReader.transformTo(new IntChunkReader(options), + (chunk, ii) -> floatCast(Integer.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + case 64: + return FloatChunkReader.transformTo(new LongChunkReader(options), + (chunk, ii) -> floatCast(Long.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static float floatCast( + final int numBytes, + boolean signed, + boolean isNull, + long value) { + if (isNull) { + // note that we widen the value without proper null handling + return QueryConstants.NULL_FLOAT; + } + if (signed) { + return QueryLanguageFunctionUtils.floatCast(value); + } + + if (numBytes == Long.BYTES) { + long lo = value & ((1L << 32) - 1); + long hi = (value >> 32) & ((1L << 32) - 1); + return ((float) hi) * 2e32f + (float) lo; + } + + // can mask in place + value &= (1L << (numBytes * Byte.SIZE)) - 1; + return QueryLanguageFunctionUtils.floatCast(value); + } + + private static ChunkReader> intToDouble( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean signed = intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return DoubleChunkReader.transformTo(new ByteChunkReader(options), + (chunk, ii) -> doubleCast(Byte.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + case 16: + return DoubleChunkReader.transformTo(new ShortChunkReader(options), + (chunk, ii) -> doubleCast(Short.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + case 32: + return DoubleChunkReader.transformTo(new IntChunkReader(options), + (chunk, ii) -> doubleCast(Integer.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + case 64: + return DoubleChunkReader.transformTo(new LongChunkReader(options), + (chunk, ii) -> doubleCast(Long.BYTES, signed, chunk.isNullAt(ii), chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static double doubleCast( + final int numBytes, + boolean signed, + boolean isNull, + long value) { + if (isNull) { + // note that we widen the value without proper null handling + return QueryConstants.NULL_DOUBLE; + } + if (signed) { + return QueryLanguageFunctionUtils.doubleCast(value); + } + + if (numBytes == Long.BYTES) { + long lo = value & ((1L << 32) - 1); + long hi = (value >> 32) & ((1L << 32) - 1); + return ((double) hi) * 2e32 + (double) lo; + } + + // can mask in place + value &= (1L << (numBytes * Byte.SIZE)) - 1; + return QueryLanguageFunctionUtils.doubleCast(value); + } + + private static ChunkReader> intToBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean unsigned = !intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return transformToObject(new ByteChunkReader(options), (chunk, ii) -> toBigDecimal(maskIfOverflow( + unsigned, Byte.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii))))); + case 16: + return transformToObject(new ShortChunkReader(options), (chunk, ii) -> toBigDecimal(maskIfOverflow( + unsigned, Short.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii))))); + case 32: + return transformToObject(new IntChunkReader(options), (chunk, ii) -> toBigDecimal(maskIfOverflow( + unsigned, Integer.BYTES, QueryLanguageFunctionUtils.longCast(chunk.get(ii))))); + case 64: + return transformToObject(new LongChunkReader(options), (chunk, ii) -> { + final BigInteger bi = maskIfOverflow(unsigned, Long.BYTES, toBigInt(chunk.get(ii))); + return bi == null ? null : new BigDecimal(bi); + }); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> intToChar( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + final boolean unsigned = !intType.getIsSigned(); + + switch (bitWidth) { + case 8: + return CharChunkReader.transformTo(new ByteChunkReader(options), + (chunk, ii) -> maskIfOverflow(unsigned, Byte.BYTES, + QueryLanguageFunctionUtils.charCast(chunk.get(ii)))); + case 16: + return new CharChunkReader(options); + case 32: + // note unsigned mappings to char will overflow short; but user has asked for this + return CharChunkReader.transformTo(new IntChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.charCast(chunk.get(ii))); + case 64: + // note unsigned mappings to short will overflow short; but user has asked for this + return CharChunkReader.transformTo(new LongChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.charCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkReader> boolToBoolean( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + return new BooleanChunkReader(); + } + + private static ChunkReader> fixedSizeBinaryToByteArray( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + final ArrowType.FixedSizeBinary fixedSizeBinary = (ArrowType.FixedSizeBinary) arrowType; + final int elementWidth = fixedSizeBinary.getByteWidth(); + return new FixedWidthChunkReader<>(elementWidth, false, options, (dataInput) -> { + final byte[] value = new byte[elementWidth]; + dataInput.readFully(value); + return value; + }); + } + + private static ChunkReader> dateToInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // see dateToLocalDate's comment for more information on wire format + final ArrowType.Date dateType = (ArrowType.Date) arrowType; + switch (dateType.getUnit()) { + case DAY: + return new IntChunkReader(options); + case MILLISECOND: + return IntChunkReader.transformTo(new LongChunkReader(options), (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_INT : (int) (value / MS_PER_DAY); + }); + default: + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); + } + } + + private static ChunkReader> dateToLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // see dateToLocalDate's comment for more information on wire format + final ArrowType.Date dateType = (ArrowType.Date) arrowType; + switch (dateType.getUnit()) { + case DAY: + return LongChunkReader.transformTo(new IntChunkReader(options), + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + case MILLISECOND: + return LongChunkReader.transformTo(new LongChunkReader(options), (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / MS_PER_DAY; + }); + default: + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); + } + } + + private static ChunkReader> dateToLocalDate( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + /* + * Date is either a 32-bit or 64-bit signed integer type representing an elapsed time since UNIX epoch + * (1970-01-01), stored in either of two units: + * + * @formatter:off + * - Milliseconds (64 bits) indicating UNIX time elapsed since the epoch (no leap seconds), where the values are + * evenly divisible by 86400000 + * - Days (32 bits) since the UNIX epoch + * @formatter:on + */ + final ArrowType.Date dateType = (ArrowType.Date) arrowType; + switch (dateType.getUnit()) { + case DAY: + return transformToObject(new IntChunkReader(options), (chunk, ii) -> { + int value = chunk.get(ii); + return value == QueryConstants.NULL_INT ? null : DateTimeUtils.epochDaysToLocalDate(value); + }); + case MILLISECOND: + return transformToObject(new LongChunkReader(options), (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG + ? null + : DateTimeUtils.epochDaysToLocalDate(value / MS_PER_DAY); + }); + default: + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); + } + } + + private static ChunkReader> intervalToDurationLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // See intervalToPeriod's comment for more information on wire format. + + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + case MONTH_DAY_NANO: + throw new IllegalArgumentException(String.format( + "Do not support %s interval to Duration as long conversion", intervalType)); + + case DAY_TIME: + return LongChunkReader + .transformTo(new FixedWidthChunkReader<>(Integer.BYTES * 2, false, options, dataInput -> { + final int days = dataInput.readInt(); + final int millis = dataInput.readInt(); + return Duration.ofDays(days).plusMillis(millis); + }), (chunk, ii) -> { + final Duration value = chunk.get(ii); + return value == null ? QueryConstants.NULL_LONG : value.toNanos(); + }); + + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static ChunkReader> intervalToDuration( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // See intervalToPeriod's comment for more information on wire format. + + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + case MONTH_DAY_NANO: + throw new IllegalArgumentException(String.format( + "Do not support %s interval to Duration conversion", intervalType)); + + case DAY_TIME: + return new FixedWidthChunkReader<>(Integer.BYTES * 2 + Long.BYTES, false, options, dataInput -> { + final int days = dataInput.readInt(); + final int millis = dataInput.readInt(); + return Duration.ofDays(days).plusMillis(millis); + }); + + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static ChunkReader> intervalToPeriod( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + /* + * A "calendar" interval which models types that don't necessarily have a precise duration without the context + * of a base timestamp (e.g. days can differ in length during day light savings time transitions). All integers + * in the types below are stored in the endianness indicated by the schema. + * + * @formatter:off + * YEAR_MONTH: + * Indicates the number of elapsed whole months, stored as 4-byte signed integers. + * + * DAY_TIME: + * Indicates the number of elapsed days and milliseconds (no leap seconds), stored as 2 contiguous 32-bit signed + * integers (8-bytes in total). + * + * MONTH_DAY_NANO: + * A triple of the number of elapsed months, days, and nanoseconds. The values are stored + * contiguously in 16-byte blocks. Months and days are encoded as 32-bit signed integers and nanoseconds is + * encoded as a 64-bit signed integer. Nanoseconds does not allow for leap seconds. + * @formatter:on + * + * Note: Period does not handle the time portion of DAY_TIME and MONTH_DAY_NANO. Arrow stores these in + * PeriodDuration pairs. + */ + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + return transformToObject(new IntChunkReader(options), (chunk, ii) -> { + int value = chunk.get(ii); + return value == QueryConstants.NULL_INT ? null : Period.ofMonths(value); + }); + case DAY_TIME: + return new FixedWidthChunkReader<>(Integer.BYTES * 2, false, options, dataInput -> { + final int days = dataInput.readInt(); + final int millis = dataInput.readInt(); + return Period.ofDays(days).plusDays(millis / MS_PER_DAY); + }); + case MONTH_DAY_NANO: + return new FixedWidthChunkReader<>(Integer.BYTES * 2 + Long.BYTES, false, options, dataInput -> { + final int months = dataInput.readInt(); + final int days = dataInput.readInt(); + final long nanos = dataInput.readLong(); + final long NANOS_PER_MS = 1_000_000; + return Period.of(0, months, days).plusDays(nanos / (MS_PER_DAY * NANOS_PER_MS)); + }); + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static ChunkReader> intervalToPeriodDuration( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo, + final ChunkReader.Options options) { + // See intervalToPeriod's comment for more information on wire format. + + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + return transformToObject(new IntChunkReader(options), (chunk, ii) -> { + int value = chunk.get(ii); + return value == QueryConstants.NULL_INT + ? null + : new PeriodDuration(Period.ofMonths(value), Duration.ZERO); + }); + case DAY_TIME: + return new FixedWidthChunkReader<>(Integer.BYTES * 2, false, options, dataInput -> { + final int days = dataInput.readInt(); + final int millis = dataInput.readInt(); + return new PeriodDuration(Period.ofDays(days), Duration.ofMillis(millis)); + }); + case MONTH_DAY_NANO: + return new FixedWidthChunkReader<>(Integer.BYTES * 2 + Long.BYTES, false, options, dataInput -> { + final int months = dataInput.readInt(); + final int days = dataInput.readInt(); + final long nanos = dataInput.readLong(); + return new PeriodDuration(Period.of(0, months, days), Duration.ofNanos(nanos)); + }); + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static BigInteger toBigInt(final long value) { + return value == QueryConstants.NULL_LONG ? null : BigInteger.valueOf(value); + } + + private static BigDecimal toBigDecimal(final long value) { + return value == QueryConstants.NULL_LONG ? null : BigDecimal.valueOf(value); + } + + @SuppressWarnings("SameParameterValue") + private static char maskIfOverflow(final boolean unsigned, final int numBytes, char value) { + if (unsigned && value != QueryConstants.NULL_CHAR) { + value &= (char) ((1L << (numBytes * 8)) - 1); + } + return value; + } + + @SuppressWarnings("SameParameterValue") + private static short maskIfOverflow(final boolean unsigned, final int numBytes, short value) { + if (unsigned && value != QueryConstants.NULL_SHORT && value < 0) { + value &= (short) ((1L << (numBytes * 8)) - 1); + } + return value; + } + + private static int maskIfOverflow(final boolean unsigned, final int numBytes, int value) { + if (unsigned && value != QueryConstants.NULL_INT && value < 0) { + value &= (int) ((1L << (numBytes * 8)) - 1); + } + return value; + } + + private static long maskIfOverflow(final boolean unsigned, final int numBytes, long value) { + if (unsigned && value != QueryConstants.NULL_LONG && value < 0) { + value &= ((1L << (numBytes * 8)) - 1); + } + return value; + } + + @SuppressWarnings("SameParameterValue") + private static BigInteger maskIfOverflow(final boolean unsigned, final int numBytes, final BigInteger value) { + if (unsigned && value != null && value.compareTo(BigInteger.ZERO) < 0) { + return value.and(BigInteger.ONE.shiftLeft(numBytes * 8).subtract(BigInteger.ONE)); + } + return value; + } + + private interface ToObjectTransformFunction> { + T get(WireChunkType wireValues, int wireOffset); + } + + private static , CR extends ChunkReader> ChunkReader> transformToObject( + final CR wireReader, + final ToObjectTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableObjectChunk::makeWritableChunk, + WritableChunk::asWritableObjectChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReadingFactory.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReadingFactory.java deleted file mode 100644 index dc1a7895aea..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkReadingFactory.java +++ /dev/null @@ -1,202 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import com.google.common.base.Charsets; -import io.deephaven.extensions.barrage.ColumnConversionMode; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.time.DateTimeUtils; -import io.deephaven.util.QueryConstants; -import io.deephaven.util.type.TypeUtils; -import io.deephaven.vector.Vector; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalTime; -import java.time.ZonedDateTime; -import java.util.Arrays; - -import static io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.MS_PER_DAY; - -/** - * JVM implementation of {@link ChunkReader.Factory}, suitable for use in Java clients and servers. This default - * implementation may not round trip flight types in a stable way, but will round trip Deephaven table definitions and - * table data. Neither of these is a required/expected property of being a Flight/Barrage/Deephaven client. - */ -public final class DefaultChunkReadingFactory implements ChunkReader.Factory { - public static final ChunkReader.Factory INSTANCE = new DefaultChunkReadingFactory(); - - @Override - public ChunkReader getReader(StreamReaderOptions options, int factor, - ChunkReader.TypeInfo typeInfo) { - // TODO (deephaven-core#5453): pass in ArrowType to enable ser/deser of single java class in multiple formats - switch (typeInfo.chunkType()) { - case Boolean: - throw new UnsupportedOperationException("Booleans are reinterpreted as bytes"); - case Char: - return new CharChunkReader(options); - case Byte: - if (typeInfo.type() == Boolean.class || typeInfo.type() == boolean.class) { - return new BooleanChunkReader(); - } - return new ByteChunkReader(options); - case Short: - return new ShortChunkReader(options); - case Int: - return new IntChunkReader(options); - case Long: - if (factor == 1) { - return new LongChunkReader(options); - } - return new LongChunkReader(options, - (long v) -> v == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : (v * factor)); - case Float: - return new FloatChunkReader(options); - case Double: - return new DoubleChunkReader(options); - case Object: - if (typeInfo.type().isArray()) { - if (typeInfo.componentType() == byte.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> VarBinaryChunkInputStreamGenerator.extractChunkFromInputStream( - is, - fieldNodeIter, - bufferInfoIter, - (buf, off, len) -> Arrays.copyOfRange(buf, off, off + len), - outChunk, outOffset, totalRows); - } else { - return new VarListChunkReader<>(options, typeInfo, this); - } - } - if (Vector.class.isAssignableFrom(typeInfo.type())) { - return new VectorChunkReader(options, typeInfo, this); - } - if (typeInfo.type() == BigInteger.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> VarBinaryChunkInputStreamGenerator.extractChunkFromInputStream( - is, - fieldNodeIter, - bufferInfoIter, - BigInteger::new, - outChunk, outOffset, totalRows); - } - if (typeInfo.type() == BigDecimal.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> VarBinaryChunkInputStreamGenerator.extractChunkFromInputStream( - is, - fieldNodeIter, - bufferInfoIter, - (final byte[] buf, final int offset, final int length) -> { - // read the int scale value as little endian, arrow's endianness. - final byte b1 = buf[offset]; - final byte b2 = buf[offset + 1]; - final byte b3 = buf[offset + 2]; - final byte b4 = buf[offset + 3]; - final int scale = b4 << 24 | (b3 & 0xFF) << 16 | (b2 & 0xFF) << 8 | (b1 & 0xFF); - return new BigDecimal(new BigInteger(buf, offset + 4, length - 4), scale); - }, - outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Instant.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Long.BYTES, options, io -> { - final long value = io.readLong(); - if (value == QueryConstants.NULL_LONG) { - return null; - } - return DateTimeUtils.epochNanosToInstant(value * factor); - }, - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == ZonedDateTime.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Long.BYTES, options, io -> { - final long value = io.readLong(); - if (value == QueryConstants.NULL_LONG) { - return null; - } - return DateTimeUtils.epochNanosToZonedDateTime( - value * factor, DateTimeUtils.timeZone()); - }, - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Byte.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Byte.BYTES, options, io -> TypeUtils.box(io.readByte()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Character.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Character.BYTES, options, io -> TypeUtils.box(io.readChar()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Double.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Double.BYTES, options, io -> TypeUtils.box(io.readDouble()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Float.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Float.BYTES, options, io -> TypeUtils.box(io.readFloat()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Integer.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Integer.BYTES, options, io -> TypeUtils.box(io.readInt()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Long.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Long.BYTES, options, io -> TypeUtils.box(io.readLong()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == Short.class) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> FixedWidthChunkInputStreamGenerator - .extractChunkFromInputStreamWithTypeConversion( - Short.BYTES, options, io -> TypeUtils.box(io.readShort()), - fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows); - } - if (typeInfo.type() == LocalDate.class) { - return new LongChunkReader(options).transform(value -> value == QueryConstants.NULL_LONG ? null - : LocalDate.ofEpochDay(value / MS_PER_DAY)); - } - if (typeInfo.type() == LocalTime.class) { - return new LongChunkReader(options).transform( - value -> value == QueryConstants.NULL_LONG ? null : LocalTime.ofNanoOfDay(value)); - } - if (typeInfo.type() == String.class || - options.columnConversionMode().equals(ColumnConversionMode.Stringify)) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> VarBinaryChunkInputStreamGenerator.extractChunkFromInputStream(is, - fieldNodeIter, - bufferInfoIter, - (buf, off, len) -> new String(buf, off, len, Charsets.UTF_8), outChunk, outOffset, - totalRows); - } - throw new UnsupportedOperationException( - "Do not yet support column conversion mode: " + options.columnConversionMode()); - default: - throw new UnsupportedOperationException(); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkWriterFactory.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkWriterFactory.java new file mode 100644 index 00000000000..34d42349a60 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DefaultChunkWriterFactory.java @@ -0,0 +1,1240 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.ByteChunk; +import io.deephaven.chunk.CharChunk; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.DoubleChunk; +import io.deephaven.chunk.FloatChunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.impl.lang.QueryLanguageFunctionUtils; +import io.deephaven.engine.table.impl.preview.ArrayPreview; +import io.deephaven.engine.table.impl.preview.DisplayWrapper; +import io.deephaven.extensions.barrage.chunk.array.ArrayExpansionKernel; +import io.deephaven.extensions.barrage.chunk.vector.VectorExpansionKernel; +import io.deephaven.extensions.barrage.util.Float16; +import io.deephaven.internal.log.LoggerFactory; +import io.deephaven.io.logger.Logger; +import io.deephaven.time.DateTimeUtils; +import io.deephaven.util.QueryConstants; +import io.deephaven.util.type.TypeUtils; +import io.deephaven.vector.Vector; +import org.apache.arrow.vector.PeriodDuration; +import org.apache.arrow.vector.types.TimeUnit; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.jetbrains.annotations.NotNull; + +import java.io.DataOutput; +import java.io.IOException; +import java.io.OutputStream; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.math.RoundingMode; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.time.ZonedDateTime; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * JVM implementation of {@link ChunkWriter.Factory}, suitable for use in Java clients and servers. This default + * implementation may not round trip flight types in a stable way, but will round trip Deephaven table definitions and + * table data. Neither of these is a required/expected property of being a Flight/Barrage/Deephaven client. + */ +public class DefaultChunkWriterFactory implements ChunkWriter.Factory { + public static final Logger log = LoggerFactory.getLogger(DefaultChunkWriterFactory.class); + public static final ChunkWriter.Factory INSTANCE = new DefaultChunkWriterFactory(); + + protected interface ChunkWriterFactory { + ChunkWriter> make( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo); + } + + private final Map, ChunkWriterFactory>> registeredFactories = + new HashMap<>(); + + protected DefaultChunkWriterFactory() { + register(ArrowType.ArrowTypeID.Timestamp, long.class, DefaultChunkWriterFactory::timestampFromLong); + register(ArrowType.ArrowTypeID.Timestamp, Instant.class, DefaultChunkWriterFactory::timestampFromInstant); + register(ArrowType.ArrowTypeID.Timestamp, ZonedDateTime.class, + DefaultChunkWriterFactory::timestampFromZonedDateTime); + register(ArrowType.ArrowTypeID.Utf8, String.class, DefaultChunkWriterFactory::utf8FromString); + register(ArrowType.ArrowTypeID.Utf8, Object.class, DefaultChunkWriterFactory::utf8FromObject); + register(ArrowType.ArrowTypeID.Utf8, ArrayPreview.class, DefaultChunkWriterFactory::utf8FromObject); + register(ArrowType.ArrowTypeID.Utf8, DisplayWrapper.class, DefaultChunkWriterFactory::utf8FromObject); + register(ArrowType.ArrowTypeID.Duration, long.class, DefaultChunkWriterFactory::durationFromLong); + register(ArrowType.ArrowTypeID.Duration, Duration.class, DefaultChunkWriterFactory::durationFromDuration); + register(ArrowType.ArrowTypeID.FloatingPoint, float.class, DefaultChunkWriterFactory::floatingPointFromFloat); + register(ArrowType.ArrowTypeID.FloatingPoint, double.class, + DefaultChunkWriterFactory::floatingPointFromDouble); + register(ArrowType.ArrowTypeID.FloatingPoint, BigDecimal.class, + DefaultChunkWriterFactory::floatingPointFromBigDecimal); + register(ArrowType.ArrowTypeID.Binary, byte[].class, DefaultChunkWriterFactory::binaryFromByteArray); + register(ArrowType.ArrowTypeID.Binary, BigInteger.class, DefaultChunkWriterFactory::binaryFromBigInt); + register(ArrowType.ArrowTypeID.Binary, BigDecimal.class, DefaultChunkWriterFactory::binaryFromBigDecimal); + register(ArrowType.ArrowTypeID.Time, long.class, DefaultChunkWriterFactory::timeFromLong); + register(ArrowType.ArrowTypeID.Time, LocalTime.class, DefaultChunkWriterFactory::timeFromLocalTime); + register(ArrowType.ArrowTypeID.Decimal, byte.class, DefaultChunkWriterFactory::decimalFromByte); + register(ArrowType.ArrowTypeID.Decimal, char.class, DefaultChunkWriterFactory::decimalFromChar); + register(ArrowType.ArrowTypeID.Decimal, short.class, DefaultChunkWriterFactory::decimalFromShort); + register(ArrowType.ArrowTypeID.Decimal, int.class, DefaultChunkWriterFactory::decimalFromInt); + register(ArrowType.ArrowTypeID.Decimal, long.class, DefaultChunkWriterFactory::decimalFromLong); + register(ArrowType.ArrowTypeID.Decimal, BigInteger.class, DefaultChunkWriterFactory::decimalFromBigInteger); + register(ArrowType.ArrowTypeID.Decimal, float.class, DefaultChunkWriterFactory::decimalFromFloat); + register(ArrowType.ArrowTypeID.Decimal, double.class, DefaultChunkWriterFactory::decimalFromDouble); + register(ArrowType.ArrowTypeID.Decimal, BigDecimal.class, DefaultChunkWriterFactory::decimalFromBigDecimal); + register(ArrowType.ArrowTypeID.Int, byte.class, DefaultChunkWriterFactory::intFromByte); + register(ArrowType.ArrowTypeID.Int, char.class, DefaultChunkWriterFactory::intFromChar); + register(ArrowType.ArrowTypeID.Int, short.class, DefaultChunkWriterFactory::intFromShort); + register(ArrowType.ArrowTypeID.Int, int.class, DefaultChunkWriterFactory::intFromInt); + register(ArrowType.ArrowTypeID.Int, long.class, DefaultChunkWriterFactory::intFromLong); + register(ArrowType.ArrowTypeID.Int, BigInteger.class, DefaultChunkWriterFactory::intFromObject); + register(ArrowType.ArrowTypeID.Int, float.class, DefaultChunkWriterFactory::intFromFloat); + register(ArrowType.ArrowTypeID.Int, double.class, DefaultChunkWriterFactory::intFromDouble); + register(ArrowType.ArrowTypeID.Int, BigDecimal.class, DefaultChunkWriterFactory::intFromObject); + register(ArrowType.ArrowTypeID.Bool, boolean.class, DefaultChunkWriterFactory::boolFromBoolean); + register(ArrowType.ArrowTypeID.Bool, Boolean.class, DefaultChunkWriterFactory::boolFromBoolean); + register(ArrowType.ArrowTypeID.Bool, byte.class, DefaultChunkWriterFactory::boolFromBoolean); + register(ArrowType.ArrowTypeID.FixedSizeBinary, byte[].class, + DefaultChunkWriterFactory::fixedSizeBinaryFromByteArray); + register(ArrowType.ArrowTypeID.Date, int.class, DefaultChunkWriterFactory::dateFromInt); + register(ArrowType.ArrowTypeID.Date, long.class, DefaultChunkWriterFactory::dateFromLong); + register(ArrowType.ArrowTypeID.Date, LocalDate.class, DefaultChunkWriterFactory::dateFromLocalDate); + register(ArrowType.ArrowTypeID.Interval, long.class, DefaultChunkWriterFactory::intervalFromDurationLong); + register(ArrowType.ArrowTypeID.Interval, Duration.class, DefaultChunkWriterFactory::intervalFromDuration); + register(ArrowType.ArrowTypeID.Interval, Period.class, DefaultChunkWriterFactory::intervalFromPeriod); + register(ArrowType.ArrowTypeID.Interval, PeriodDuration.class, + DefaultChunkWriterFactory::intervalFromPeriodDuration); + } + + @Override + public > ChunkWriter newWriter( + @NotNull final ChunkReader.TypeInfo typeInfo) { + // TODO (deephaven/deephaven-core#6033): Run-End Support + // TODO (deephaven/deephaven-core#6034): Dictionary Support + + final Field field = Field.convertField(typeInfo.arrowField()); + + final ArrowType.ArrowTypeID typeId = field.getType().getTypeID(); + final boolean isSpecialType = DefaultChunkReaderFactory.SPECIAL_TYPES.contains(typeId); + + // Note we do not support these as they require 64-bit offsets: + if (typeId == ArrowType.ArrowTypeID.LargeUtf8 + || typeId == ArrowType.ArrowTypeID.LargeBinary + || typeId == ArrowType.ArrowTypeID.LargeList) { + throw new UnsupportedOperationException(String.format( + "No support for 64-bit offsets to map arrow type %s from %s.", + field.getType().toString(), + typeInfo.type().getCanonicalName())); + } + + final Map, ChunkWriterFactory> knownWriters = registeredFactories.get(typeId); + if (knownWriters == null && !isSpecialType) { + throw new UnsupportedOperationException(String.format( + "No known ChunkWriter for arrow type %s from %s.", + field.getType().toString(), + typeInfo.type().getCanonicalName())); + } + + final ChunkWriterFactory chunkWriterFactory = knownWriters == null ? null : knownWriters.get(typeInfo.type()); + if (chunkWriterFactory != null) { + // noinspection unchecked + final ChunkWriter writer = (ChunkWriter) chunkWriterFactory.make(field.getType(), typeInfo); + if (writer != null) { + return writer; + } + } else if (!isSpecialType) { + throw new UnsupportedOperationException(String.format( + "No known ChunkWriter for arrow type %s from %s. Supported types: %s", + field.getType().toString(), + typeInfo.type().getCanonicalName(), + knownWriters.keySet().stream().map(Object::toString).collect(Collectors.joining(", ")))); + } + + if (typeId == ArrowType.ArrowTypeID.Null) { + return new NullChunkWriter<>(); + } + + if (typeId == ArrowType.ArrowTypeID.List + || typeId == ArrowType.ArrowTypeID.FixedSizeList) { + + // TODO (deephaven/deephaven-core#5947): Add SPARSE branch for ListView + int fixedSizeLength = 0; + final ListChunkReader.Mode mode; + if (typeId == ArrowType.ArrowTypeID.List) { + mode = ListChunkReader.Mode.DENSE; + } else { + mode = ListChunkReader.Mode.FIXED; + fixedSizeLength = ((ArrowType.FixedSizeList) field.getType()).getListSize(); + } + + final ChunkReader.TypeInfo componentTypeInfo; + final boolean useVectorKernels = Vector.class.isAssignableFrom(typeInfo.type()); + if (useVectorKernels) { + final Class componentType = + VectorExpansionKernel.getComponentType(typeInfo.type(), typeInfo.componentType()); + componentTypeInfo = new ChunkReader.TypeInfo( + componentType, + componentType.getComponentType(), + typeInfo.arrowField().children(0)); + } else if (typeInfo.type().isArray()) { + final Class componentType = typeInfo.componentType(); + // noinspection DataFlowIssue + componentTypeInfo = new ChunkReader.TypeInfo( + componentType, + componentType.getComponentType(), + typeInfo.arrowField().children(0)); + } else { + throw new UnsupportedOperationException(String.format( + "No known ChunkWriter for arrow type %s from %s. Expected destination type to be an array.", + field.getType().toString(), + typeInfo.type().getCanonicalName())); + } + + final ChunkType chunkType = ListChunkReader.getChunkTypeFor(componentTypeInfo.type()); + final ExpansionKernel kernel; + if (useVectorKernels) { + kernel = VectorExpansionKernel.makeExpansionKernel(chunkType, componentTypeInfo.type()); + } else { + kernel = ArrayExpansionKernel.makeExpansionKernel(chunkType, componentTypeInfo.type()); + } + final ChunkWriter> componentWriter = newWriter(componentTypeInfo); + + // noinspection unchecked + return (ChunkWriter) new ListChunkWriter<>(mode, fixedSizeLength, kernel, componentWriter); + } + + if (typeId == ArrowType.ArrowTypeID.Map) { + // TODO: can user supply collector? + final Field structField = field.getChildren().get(0); + final Field keyField = structField.getChildren().get(0); + final Field valueField = structField.getChildren().get(1); + + // TODO NATE NOCOMMIT: implement + } + + if (typeId == ArrowType.ArrowTypeID.Struct) { + // TODO: expose transformer API of Map> -> T + // TODO NATE NOCOMMIT: implement + } + + if (typeId == ArrowType.ArrowTypeID.Union) { + final ArrowType.Union unionType = (ArrowType.Union) field.getType(); + switch (unionType.getMode()) { + case Sparse: + // TODO NATE NOCOMMIT: implement + break; + case Dense: + // TODO NATE NOCOMMIT: implement + break; + default: + throw new IllegalArgumentException("Unexpected union mode: " + unionType.getMode()); + } + } + + throw new UnsupportedOperationException(String.format( + "No known ChunkWriter for arrow type %s from %s. Arrow type supports: %s", + field.getType().toString(), + typeInfo.type().getCanonicalName(), + knownWriters == null ? "none" + : knownWriters.keySet().stream() + .map(Object::toString) + .collect(Collectors.joining(", ")))); + } + + protected void register( + final ArrowType.ArrowTypeID arrowType, + final Class deephavenType, + final ChunkWriterFactory chunkWriterFactory) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(deephavenType, chunkWriterFactory); + + // if primitive automatically register the boxed version of this mapping, too + if (deephavenType == byte.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Byte.class, (at, typeInfo) -> new ByteChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } else if (deephavenType == short.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Short.class, (at, typeInfo) -> new ShortChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } else if (deephavenType == int.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Integer.class, (at, typeInfo) -> new IntChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } else if (deephavenType == long.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Long.class, (at, typeInfo) -> new LongChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } else if (deephavenType == char.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Character.class, (at, typeInfo) -> new CharChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } else if (deephavenType == float.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Float.class, (at, typeInfo) -> new FloatChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } else if (deephavenType == double.class) { + registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) + .put(Double.class, (at, typeInfo) -> new DoubleChunkWriter>( + ObjectChunk::getEmptyChunk, (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)))); + } + } + + private static long factorForTimeUnit(final TimeUnit unit) { + switch (unit) { + case NANOSECOND: + return 1; + case MICROSECOND: + return 1000; + case MILLISECOND: + return 1000 * 1000L; + case SECOND: + return 1000 * 1000 * 1000L; + default: + throw new IllegalArgumentException("Unexpected time unit value: " + unit); + } + } + + private static ChunkWriter> timestampFromLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Timestamp tsType = (ArrowType.Timestamp) arrowType; + final long factor = factorForTimeUnit(tsType.getUnit()); + return new LongChunkWriter<>(LongChunk::getEmptyChunk, (Chunk source, int offset) -> { + // unfortunately we do not know whether ReinterpretUtils can convert the column source to longs or not + if (source instanceof LongChunk) { + final long value = source.asLongChunk().get(offset); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; + } + + final ZonedDateTime value = source.asObjectChunk().get(offset); + return value == null ? QueryConstants.NULL_LONG : DateTimeUtils.epochNanos(value) / factor; + }); + } + + private static ChunkWriter> timestampFromInstant( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final long factor = factorForTimeUnit(((ArrowType.Timestamp) arrowType).getUnit()); + return new LongChunkWriter<>(ObjectChunk::getEmptyChunk, (source, offset) -> { + final Instant value = source.get(offset); + return value == null ? QueryConstants.NULL_LONG : DateTimeUtils.epochNanos(value) / factor; + }); + } + + private static ChunkWriter> timestampFromZonedDateTime( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Timestamp tsType = (ArrowType.Timestamp) arrowType; + final long factor = factorForTimeUnit(tsType.getUnit()); + return new LongChunkWriter<>(ObjectChunk::getEmptyChunk, (source, offset) -> { + final ZonedDateTime value = source.get(offset); + return value == null ? QueryConstants.NULL_LONG : DateTimeUtils.epochNanos(value) / factor; + }); + } + + private static ChunkWriter> utf8FromString( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + return new VarBinaryChunkWriter<>((out, item) -> out.write(item.getBytes(StandardCharsets.UTF_8))); + } + + private static ChunkWriter> utf8FromObject( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + return new VarBinaryChunkWriter<>((out, item) -> out.write(item.toString().getBytes(StandardCharsets.UTF_8))); + } + + private static ChunkWriter> durationFromLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final long factor = factorForTimeUnit(((ArrowType.Duration) arrowType).getUnit()); + return factor == 1 + ? LongChunkWriter.INSTANCE + : new LongChunkWriter<>(LongChunk::getEmptyChunk, (source, offset) -> { + final long value = source.get(offset); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; + }); + } + + private static ChunkWriter> durationFromDuration( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final long factor = factorForTimeUnit(((ArrowType.Duration) arrowType).getUnit()); + return new LongChunkWriter<>(ObjectChunk::getEmptyChunk, (source, offset) -> { + final Duration value = source.get(offset); + return value == null ? QueryConstants.NULL_LONG : value.toNanos() / factor; + }); + } + + private static ChunkWriter> floatingPointFromFloat( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) arrowType; + switch (fpType.getPrecision()) { + case HALF: + return new ShortChunkWriter<>(FloatChunk::getEmptyChunk, (source, offset) -> { + final double value = source.get(offset); + return value == QueryConstants.NULL_FLOAT + ? QueryConstants.NULL_SHORT + : Float16.toFloat16((float) value); + }); + + case SINGLE: + return FloatChunkWriter.INSTANCE; + + case DOUBLE: + return new DoubleChunkWriter<>(FloatChunk::getEmptyChunk, + (source, offset) -> QueryLanguageFunctionUtils.doubleCast(source.get(offset))); + + default: + throw new IllegalArgumentException("Unexpected floating point precision: " + fpType.getPrecision()); + } + } + + private static ChunkWriter> floatingPointFromDouble( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) arrowType; + switch (fpType.getPrecision()) { + case HALF: + return new ShortChunkWriter<>(DoubleChunk::getEmptyChunk, (source, offset) -> { + final double value = source.get(offset); + return value == QueryConstants.NULL_DOUBLE + ? QueryConstants.NULL_SHORT + : Float16.toFloat16((float) value); + }); + + case SINGLE: + return new FloatChunkWriter<>(DoubleChunk::getEmptyChunk, + (source, offset) -> QueryLanguageFunctionUtils.floatCast(source.get(offset))); + case DOUBLE: + return DoubleChunkWriter.INSTANCE; + + default: + throw new IllegalArgumentException("Unexpected floating point precision: " + fpType.getPrecision()); + } + } + + private static ChunkWriter> floatingPointFromBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) arrowType; + switch (fpType.getPrecision()) { + case HALF: + return new ShortChunkWriter<>(ObjectChunk::getEmptyChunk, (source, offset) -> { + final BigDecimal value = source.get(offset); + return value == null + ? QueryConstants.NULL_SHORT + : Float16.toFloat16(value.floatValue()); + }); + + case SINGLE: + return new FloatChunkWriter<>(ObjectChunk::getEmptyChunk, + (source, offset) -> QueryLanguageFunctionUtils.floatCast(source.get(offset))); + + case DOUBLE: + return new DoubleChunkWriter<>(ObjectChunk::getEmptyChunk, + (source, offset) -> QueryLanguageFunctionUtils.doubleCast(source.get(offset))); + + default: + throw new IllegalArgumentException("Unexpected floating point precision: " + fpType.getPrecision()); + } + } + + private static ChunkWriter> binaryFromByteArray( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + return new VarBinaryChunkWriter<>(OutputStream::write); + } + + private static ChunkWriter> binaryFromBigInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + return new VarBinaryChunkWriter<>((out, item) -> out.write(item.toByteArray())); + } + + private static ChunkWriter> binaryFromBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + return new VarBinaryChunkWriter<>((out, item) -> { + final BigDecimal normal = item.stripTrailingZeros(); + final int v = normal.scale(); + // Write as little endian, arrow endianness. + out.write(0xFF & v); + out.write(0xFF & (v >> 8)); + out.write(0xFF & (v >> 16)); + out.write(0xFF & (v >> 24)); + out.write(normal.unscaledValue().toByteArray()); + }); + } + + private static ChunkWriter> timeFromLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + // See timeFromLocalTime's comment for more information on wire format. + final ArrowType.Time timeType = (ArrowType.Time) arrowType; + final int bitWidth = timeType.getBitWidth(); + final long factor = factorForTimeUnit(timeType.getUnit()); + switch (bitWidth) { + case 32: + return new IntChunkWriter<>(LongChunk::getEmptyChunk, (chunk, ii) -> { + // note: do math prior to truncation + long value = chunk.get(ii); + value = value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; + return QueryLanguageFunctionUtils.intCast(value); + }); + + case 64: + return new LongChunkWriter<>(LongChunk::getEmptyChunk, (chunk, ii) -> { + long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; + }); + + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> timeFromLocalTime( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + /* + * Time is either a 32-bit or 64-bit signed integer type representing an elapsed time since midnight, stored in + * either of four units: seconds, milliseconds, microseconds or nanoseconds. + * + * The integer `bitWidth` depends on the `unit` and must be one of the following: + * @formatter:off + * - SECOND and MILLISECOND: 32 bits + * - MICROSECOND and NANOSECOND: 64 bits + * @formatter:on + * + * The allowed values are between 0 (inclusive) and 86400 (=24*60*60) seconds (exclusive), adjusted for the time + * unit (for example, up to 86400000 exclusive for the MILLISECOND unit). This definition doesn't allow for leap + * seconds. Time values from measurements with leap seconds will need to be corrected when ingesting into Arrow + * (for example by replacing the value 86400 with 86399). + */ + + final ArrowType.Time timeType = (ArrowType.Time) arrowType; + final int bitWidth = timeType.getBitWidth(); + final long factor = factorForTimeUnit(timeType.getUnit()); + switch (bitWidth) { + case 32: + return new IntChunkWriter<>(ObjectChunk::getEmptyChunk, (chunk, ii) -> { + // note: do math prior to truncation + final LocalTime lt = chunk.get(ii); + final long value = lt == null ? QueryConstants.NULL_LONG : lt.toNanoOfDay() / factor; + return QueryLanguageFunctionUtils.intCast(value); + }); + + case 64: + return new LongChunkWriter<>(ObjectChunk::getEmptyChunk, (chunk, ii) -> { + final LocalTime lt = chunk.get(ii); + return lt == null ? QueryConstants.NULL_LONG : lt.toNanoOfDay() / factor; + }); + + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> decimalFromByte( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(ByteChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + byte value = chunk.get(offset); + if (value == QueryConstants.NULL_BYTE) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromChar( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(CharChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + char value = chunk.get(offset); + if (value == QueryConstants.NULL_CHAR) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromShort( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(ShortChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + short value = chunk.get(offset); + if (value == QueryConstants.NULL_SHORT) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(IntChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + int value = chunk.get(offset); + if (value == QueryConstants.NULL_INT) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(LongChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + long value = chunk.get(offset); + if (value == QueryConstants.NULL_LONG) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromBigInteger( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + BigInteger value = chunk.get(offset); + if (value == null) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, new BigDecimal(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromFloat( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(FloatChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + float value = chunk.get(offset); + if (value == QueryConstants.NULL_FLOAT) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromDouble( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(DoubleChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + double value = chunk.get(offset); + if (value == QueryConstants.NULL_DOUBLE) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, BigDecimal.valueOf(value), byteWidth, scale, truncationMask, nullValue); + }); + } + + private static ChunkWriter> decimalFromBigDecimal( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Decimal decimalType = (ArrowType.Decimal) arrowType; + final int byteWidth = decimalType.getBitWidth() / 8; + final int scale = decimalType.getScale(); + final byte[] nullValue = new byte[byteWidth]; + // note that BigInteger's byte array requires one sign bit; note we negate so the BigInteger#and keeps sign + final BigInteger truncationMask = BigInteger.ONE.shiftLeft(byteWidth * 8 - 1) + .subtract(BigInteger.ONE) + .negate(); + + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, byteWidth, false, (out, chunk, offset) -> { + BigDecimal value = chunk.get(offset); + if (value == null) { + out.write(nullValue); + return; + } + + writeBigDecimal(out, value, byteWidth, scale, truncationMask, nullValue); + }); + } + + private static void writeBigDecimal( + @NotNull final DataOutput output, + @NotNull BigDecimal value, + final int byteWidth, + final int scale, + @NotNull final BigInteger truncationMask, + final byte @NotNull [] nullValue) throws IOException { + if (value.scale() != scale) { + value = value.setScale(scale, RoundingMode.HALF_UP); + } + + byte[] bytes = value.unscaledValue().and(truncationMask).toByteArray(); + int numZeroBytes = byteWidth - bytes.length; + Assert.geqZero(numZeroBytes, "numZeroBytes"); + if (numZeroBytes > 0) { + output.write(nullValue, 0, numZeroBytes); + } + output.write(bytes); + } + + private static ChunkWriter> intFromByte( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return ByteChunkWriter.INSTANCE; + case 16: + return new ShortChunkWriter<>(ByteChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return new IntChunkWriter<>(ByteChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return new LongChunkWriter<>(ByteChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromShort( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(ShortChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return ShortChunkWriter.INSTANCE; + case 32: + return new IntChunkWriter<>(ShortChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return new LongChunkWriter<>(ShortChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(IntChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return new ShortChunkWriter<>(IntChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return IntChunkWriter.INSTANCE; + case 64: + return new LongChunkWriter<>(IntChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(LongChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return new ShortChunkWriter<>(LongChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return new IntChunkWriter<>(LongChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return LongChunkWriter.INSTANCE; + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromObject( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(ObjectChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return new ShortChunkWriter<>(ObjectChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return new IntChunkWriter<>(ObjectChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return new LongChunkWriter<>(ObjectChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromChar( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(CharChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return new ShortChunkWriter<>(CharChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return new IntChunkWriter<>(CharChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return new LongChunkWriter<>(CharChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromFloat( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(FloatChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return new ShortChunkWriter<>(FloatChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return new IntChunkWriter<>(FloatChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return new LongChunkWriter<>(FloatChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> intFromDouble( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.Int intType = (ArrowType.Int) arrowType; + final int bitWidth = intType.getBitWidth(); + + switch (bitWidth) { + case 8: + return new ByteChunkWriter<>(DoubleChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii))); + case 16: + return new ShortChunkWriter<>(DoubleChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii))); + case 32: + return new IntChunkWriter<>(DoubleChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + case 64: + return new LongChunkWriter<>(DoubleChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii))); + default: + throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); + } + } + + private static ChunkWriter> boolFromBoolean( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + return new BooleanChunkWriter(); + } + + private static ChunkWriter> fixedSizeBinaryFromByteArray( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + final ArrowType.FixedSizeBinary fixedSizeBinary = (ArrowType.FixedSizeBinary) arrowType; + final int elementWidth = fixedSizeBinary.getByteWidth(); + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, elementWidth, false, + (out, chunk, offset) -> { + final byte[] data = chunk.get(offset); + if (data.length != elementWidth) { + throw new IllegalArgumentException(String.format( + "Expected fixed size binary of %d bytes, but got %d bytes when serializing %s", + elementWidth, data.length, typeInfo.type().getCanonicalName())); + } + out.write(data); + }); + } + + private static ChunkWriter> dateFromInt( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + // see dateFromLocalDate's comment for more information on wire format + final ArrowType.Date dateType = (ArrowType.Date) arrowType; + switch (dateType.getUnit()) { + case DAY: + return new IntChunkWriter<>(IntChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + + case MILLISECOND: + return new LongChunkWriter<>(IntChunk::getEmptyChunk, (chunk, ii) -> { + final long value = QueryLanguageFunctionUtils.longCast(chunk.get(ii)); + return value == QueryConstants.NULL_LONG + ? QueryConstants.NULL_LONG + : (value * ChunkWriter.MS_PER_DAY); + }); + default: + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); + } + } + + private static ChunkWriter> dateFromLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + // see dateFromLocalDate's comment for more information on wire format + final ArrowType.Date dateType = (ArrowType.Date) arrowType; + switch (dateType.getUnit()) { + case DAY: + return new IntChunkWriter<>(LongChunk::getEmptyChunk, + (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii))); + + case MILLISECOND: + return new LongChunkWriter<>(LongChunk::getEmptyChunk, (chunk, ii) -> { + final long value = chunk.get(ii); + return value == QueryConstants.NULL_LONG + ? QueryConstants.NULL_LONG + : (value * ChunkWriter.MS_PER_DAY); + }); + default: + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); + } + } + + private static ChunkWriter> dateFromLocalDate( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + /* + * Date is either a 32-bit or 64-bit signed integer type representing an elapsed time since UNIX epoch + * (1970-01-01), stored in either of two units: + * + * @formatter:off + * - Milliseconds (64 bits) indicating UNIX time elapsed since the epoch (no leap seconds), where the values are + * evenly divisible by 86400000 + * - Days (32 bits) since the UNIX epoch + * @formatter:on + */ + + final ArrowType.Date dateType = (ArrowType.Date) arrowType; + switch (dateType.getUnit()) { + case DAY: + return new IntChunkWriter<>(ObjectChunk::getEmptyChunk, (chunk, ii) -> { + final LocalDate value = chunk.get(ii); + return value == null ? QueryConstants.NULL_INT : (int) value.toEpochDay(); + }); + case MILLISECOND: + return new LongChunkWriter<>(ObjectChunk::getEmptyChunk, (chunk, ii) -> { + final LocalDate value = chunk.get(ii); + return value == null ? QueryConstants.NULL_LONG : value.toEpochDay() * ChunkWriter.MS_PER_DAY; + }); + default: + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); + } + } + + private static ChunkWriter> intervalFromDurationLong( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + // See intervalFromPeriod's comment for more information on wire format. + + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + case MONTH_DAY_NANO: + throw new IllegalArgumentException(String.format( + "Do not support %s interval from duration as long conversion", intervalType)); + + case DAY_TIME: + return new FixedWidthChunkWriter<>(LongChunk::getEmptyChunk, Integer.BYTES * 2, false, + (out, source, offset) -> { + final long value = source.get(offset); + if (value == QueryConstants.NULL_LONG) { + out.writeInt(0); + out.writeInt(0); + } else { + // days then millis + out.writeInt((int) (value / ChunkWriter.NS_PER_DAY)); + out.writeInt((int) ((value % ChunkWriter.NS_PER_DAY) / ChunkWriter.NS_PER_MS)); + } + }); + + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static ChunkWriter> intervalFromDuration( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + // See intervalFromPeriod's comment for more information on wire format. + + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + case MONTH_DAY_NANO: + throw new IllegalArgumentException(String.format( + "Do not support %s interval from duration as long conversion", intervalType)); + + case DAY_TIME: + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, Integer.BYTES * 2, false, + (out, source, offset) -> { + final Duration value = source.get(offset); + if (value == null) { + out.writeInt(0); + out.writeInt(0); + } else { + // days then millis + out.writeInt((int) value.toDays()); + out.writeInt((int) (value.getNano() / ChunkWriter.NS_PER_MS)); + } + }); + + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static ChunkWriter> intervalFromPeriod( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + /* + * A "calendar" interval which models types that don't necessarily have a precise duration without the context + * of a base timestamp (e.g. days can differ in length during day light savings time transitions). All integers + * in the types below are stored in the endianness indicated by the schema. + * + * @formatter:off + * YEAR_MONTH: + * Indicates the number of elapsed whole months, stored as 4-byte signed integers. + * + * DAY_TIME: + * Indicates the number of elapsed days and milliseconds (no leap seconds), stored as 2 contiguous 32-bit signed + * integers (8-bytes in total). + * + * MONTH_DAY_NANO: + * A triple of the number of elapsed months, days, and nanoseconds. The values are stored + * contiguously in 16-byte blocks. Months and days are encoded as 32-bit signed integers and nanoseconds is + * encoded as a 64-bit signed integer. Nanoseconds does not allow for leap seconds. + * @formatter:on + * + * Note: Period does not handle the time portion of DAY_TIME and MONTH_DAY_NANO. Arrow stores these in + * PeriodDuration pairs. + */ + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + return new IntChunkWriter<>(ObjectChunk::getEmptyChunk, (chunk, ii) -> { + final Period value = chunk.get(ii); + return value == null ? QueryConstants.NULL_INT : value.getMonths() + value.getYears() * 12; + }); + case DAY_TIME: + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, Integer.BYTES * 2, false, + (out, chunk, offset) -> { + final Period value = chunk.get(offset); + if (value == null) { + out.writeInt(0); + out.writeInt(0); + } else { + // days then millis + out.writeInt(value.getDays()); + out.writeInt(0); + } + }); + case MONTH_DAY_NANO: + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, Integer.BYTES * 2 + Long.BYTES, false, + (out, chunk, offset) -> { + final Period value = chunk.get(offset); + if (value == null) { + out.writeInt(0); + out.writeInt(0); + out.writeLong(0); + } else { + out.writeInt(value.getMonths() + value.getYears() * 12); + out.writeInt(value.getDays()); + out.writeLong(0); + } + }); + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } + + private static ChunkWriter> intervalFromPeriodDuration( + final ArrowType arrowType, + final ChunkReader.TypeInfo typeInfo) { + // See intervalToPeriod's comment for more information on wire format. + + final ArrowType.Interval intervalType = (ArrowType.Interval) arrowType; + switch (intervalType.getUnit()) { + case YEAR_MONTH: + return new IntChunkWriter<>(ObjectChunk::getEmptyChunk, (chunk, ii) -> { + final Period value = chunk.get(ii).getPeriod(); + return value == null ? QueryConstants.NULL_INT : value.getMonths() + value.getYears() * 12; + }); + case DAY_TIME: + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, Integer.BYTES * 2, false, + (out, chunk, offset) -> { + final PeriodDuration value = chunk.get(offset); + if (value == null) { + out.writeInt(0); + out.writeInt(0); + } else { + // days then millis + out.writeInt(value.getPeriod().getDays()); + out.writeInt(value.getDuration().getNano()); + } + }); + case MONTH_DAY_NANO: + return new FixedWidthChunkWriter<>(ObjectChunk::getEmptyChunk, Integer.BYTES * 2 + Long.BYTES, false, + (out, chunk, offset) -> { + final PeriodDuration value = chunk.get(offset); + if (value == null) { + out.writeInt(0); + out.writeInt(0); + out.writeLong(0); + } else { + final Period period = value.getPeriod(); + out.writeInt(period.getMonths() + period.getYears() * 12); + out.writeInt(period.getDays()); + out.writeLong(value.getDuration().getNano()); + } + }); + default: + throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java deleted file mode 100644 index a0046b67edb..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java +++ /dev/null @@ -1,162 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkInputStreamGenerator and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off -package io.deephaven.extensions.barrage.chunk; - -import java.util.function.ToDoubleFunction; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.DoubleChunk; -import io.deephaven.chunk.WritableDoubleChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class DoubleChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "DoubleChunkInputStreamGenerator"; - - public static DoubleChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static DoubleChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToDoubleFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableDoubleChunk outChunk = WritableDoubleChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsDouble(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new DoubleChunkInputStreamGenerator(outChunk, Double.BYTES, rowOffset); - } - - DoubleChunkInputStreamGenerator(final DoubleChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new DoubleChunkInputStream(options, subset); - } - - private class DoubleChunkInputStream extends BaseChunkInputStream { - private DoubleChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_DOUBLE) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_DOUBLE) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final double val = chunk.get((int) row); - dos.writeDouble(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("DoubleChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkReader.java index 39059f29a2f..4ae5b478b6f 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkReader.java @@ -1,83 +1,63 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkReader and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off package io.deephaven.extensions.barrage.chunk; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; +import io.deephaven.extensions.barrage.util.Float16; +import io.deephaven.util.QueryConstants; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.apache.arrow.flatbuf.Precision; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; -import java.util.function.Function; -import java.util.function.IntFunction; - -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -public class DoubleChunkReader implements ChunkReader { +public class DoubleChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "DoubleChunkReader"; - private final StreamReaderOptions options; - private final DoubleConversion conversion; - - @FunctionalInterface - public interface DoubleConversion { - double apply(double in); - DoubleConversion IDENTITY = (double a) -> a; + public interface ToDoubleTransformFunction> { + double get(WireChunkType wireValues, int wireOffset); } - public DoubleChunkReader(StreamReaderOptions options) { - this(options, DoubleConversion.IDENTITY); - } - - public DoubleChunkReader(StreamReaderOptions options, DoubleConversion conversion) { - this.options = options; - this.conversion = conversion; + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToDoubleTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableDoubleChunk::makeWritableChunk, + WritableChunk::asWritableDoubleChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); } - public ChunkReader transform(Function transform) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { - try (final WritableDoubleChunk inner = DoubleChunkReader.this.readChunk( - fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { + private final short precisionFlatbufId; + private final ChunkReader.Options options; - final WritableObjectChunk chunk = castOrCreateChunk( - outChunk, - Math.max(totalRows, inner.size()), - WritableObjectChunk::makeWritableChunk, - WritableChunk::asWritableObjectChunk); - - if (outChunk == null) { - // if we're not given an output chunk then we better be writing at the front of the new one - Assert.eqZero(outOffset, "outOffset"); - } - - for (int ii = 0; ii < inner.size(); ++ii) { - double value = inner.get(ii); - chunk.set(outOffset + ii, transform.apply(value)); - } - - return chunk; - } - }; + public DoubleChunkReader( + final short precisionFlatbufId, + final ChunkReader.Options options) { + this.precisionFlatbufId = precisionFlatbufId; + this.options = options; } @Override - public WritableDoubleChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableDoubleChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -93,9 +73,6 @@ public WritableDoubleChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -114,9 +91,9 @@ public WritableDoubleChunk readChunk(Iterator readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( - final DoubleConversion conversion, + final short precisionFlatbufId, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableDoubleChunk chunk, final int offset) throws IOException { - if (conversion == DoubleConversion.IDENTITY) { - for (int ii = 0; ii < nodeInfo.numElements; ++ii) { - chunk.set(offset + ii, is.readDouble()); - } - } else { - for (int ii = 0; ii < nodeInfo.numElements; ++ii) { - final double in = is.readDouble(); - final double out = in == NULL_DOUBLE ? in : conversion.apply(in); - chunk.set(offset + ii, out); - } + switch (precisionFlatbufId) { + case Precision.HALF: + throw new IllegalStateException("Cannot use Deephaven nulls with half-precision floats"); + case Precision.SINGLE: + for (int ii = 0; ii < nodeInfo.numElements; ++ii) { + final float v = is.readFloat(); + chunk.set(offset + ii, v == QueryConstants.NULL_FLOAT ? QueryConstants.NULL_DOUBLE : v); + } + break; + case Precision.DOUBLE: + for (int ii = 0; ii < nodeInfo.numElements; ++ii) { + chunk.set(offset + ii, is.readDouble()); + } + break; + default: + throw new IllegalStateException("Unsupported floating point precision: " + precisionFlatbufId); } } + @FunctionalInterface + private interface DoubleSupplier { + double next() throws IOException; + } + + private static double doubleCast(float a) { + return a == QueryConstants.NULL_FLOAT ? QueryConstants.NULL_DOUBLE : (double) a; + } + private static void useValidityBuffer( - final DoubleConversion conversion, + final short precisionFlatbufId, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableDoubleChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { @@ -173,18 +152,37 @@ private static void useValidityBuffer( int ei = 0; int pendingSkips = 0; + final int elementSize; + final DoubleSupplier supplier; + switch (precisionFlatbufId) { + case Precision.HALF: + elementSize = Short.BYTES; + supplier = () -> Float16.toFloat(is.readShort()); + break; + case Precision.SINGLE: + elementSize = Float.BYTES; + supplier = () -> doubleCast(is.readFloat()); + break; + case Precision.DOUBLE: + elementSize = Double.BYTES; + supplier = is::readDouble; + break; + default: + throw new IllegalStateException("Unsupported floating point precision: " + precisionFlatbufId); + } + for (int vi = 0; vi < numValidityWords; ++vi) { int bitsLeftInThisWord = Math.min(64, numElements - vi * 64); long validityWord = isValid.get(vi); do { if ((validityWord & 1) == 1) { if (pendingSkips > 0) { - is.skipBytes(pendingSkips * Double.BYTES); + is.skipBytes(pendingSkips * elementSize); chunk.fillWithNullValue(offset + ei, pendingSkips); ei += pendingSkips; pendingSkips = 0; } - chunk.set(offset + ei++, conversion.apply(is.readDouble())); + chunk.set(offset + ei++, supplier.next()); validityWord >>= 1; bitsLeftInThisWord--; } else { @@ -197,7 +195,7 @@ private static void useValidityBuffer( } if (pendingSkips > 0) { - is.skipBytes(pendingSkips * Double.BYTES); + is.skipBytes(pendingSkips * elementSize); chunk.fillWithNullValue(offset + ei, pendingSkips); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkWriter.java new file mode 100644 index 00000000000..c590011ac42 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkWriter.java @@ -0,0 +1,101 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkWriter and run "./gradlew replicateBarrageUtils" to regenerate +// +// @formatter:off +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.DoubleChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class DoubleChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "DoubleChunkWriter"; + public static final DoubleChunkWriter> INSTANCE = new DoubleChunkWriter<>( + DoubleChunk::getEmptyChunk, DoubleChunk::get); + + @FunctionalInterface + public interface ToDoubleTransformFunction> { + double get(SourceChunkType sourceValues, int offset); + } + + private final ToDoubleTransformFunction transform; + + public DoubleChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToDoubleTransformFunction transform) { + super(emptyChunkSupplier, Double.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new DoubleChunkInputStream(context, subset, options); + } + + private class DoubleChunkInputStream extends BaseChunkInputStream> { + private DoubleChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeDouble(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ExpansionKernel.java new file mode 100644 index 00000000000..8c18bb552e2 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ExpansionKernel.java @@ -0,0 +1,103 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.IntChunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; +import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +public interface ExpansionKernel { + + /** + * This expands the source from a {@code V} per element to a flat {@code T} per element. The kernel records the + * number of consecutive elements that belong to a row in {@code offsetDest}. The returned chunk is owned by the + * caller. + *

+ * If a non-zero {@code fixedSizeLength} is provided, then each row will be truncated or null-appended as + * appropriate to match the fixed size. + * + * @param source the source chunk of V to expand + * @param fixedSizeLength the length of each array, which is fixed for all rows + * @param offsetDest the destination IntChunk for which {@code dest.get(i + 1) - dest.get(i)} is equivalent to + * {@code source.get(i).length} + * @return an unrolled/flattened chunk of T + */ + default WritableChunk expand( + @NotNull ObjectChunk source, + int fixedSizeLength, + @Nullable WritableIntChunk offsetDest) { + // TODO NATE NOCOMMII: implement fixed size list length restrictions! + return expand(source, offsetDest); + } + + // TODO NATE NOCOMMIT: THIS METHOD DOES NOT GET TO STAY + WritableChunk expand( + @NotNull ObjectChunk source, + @Nullable WritableIntChunk offsetDest); + + /** + * This contracts the source from a pair of {@code LongChunk} and {@code Chunk} and produces a {@code Chunk}. + * The returned chunk is owned by the caller. + *

+ * The method of determining the length of each row is determined by whether {@code offsets} and {@code lengths} are + * {@code null} or not. If offsets is {@code null}, then the length of each row is assumed to be + * {@code sizePerElement}. If {@code lengths} is {@code null}, then the length of each row is determined by adjacent + * elements in {@code offsets}. If both are non-{@code null}, then the length of each row is determined by + * {@code lengths}. + * + * @param source the source chunk of T to contract + * @param sizePerElement the length of each array, which is fixed for all rows + * @param offsets the source IntChunk to determine the start location of each row + * @param lengths the source IntChunk to determine the length of each row + * @param outChunk the returned chunk from an earlier record batch + * @param outOffset the offset to start writing into {@code outChunk} + * @param totalRows the total known rows for this column; if known (else 0) + * @return a result chunk of {@code V} + */ + WritableObjectChunk contract( + @NotNull Chunk source, + int sizePerElement, + @Nullable IntChunk offsets, + @Nullable IntChunk lengths, + @Nullable WritableChunk outChunk, + int outOffset, + int totalRows); + + /** + * This computes the length of a given index depending on whether this is an Arrow FixedSizeList, List, or ListView. + *

+ * If {@code offsets} is {@code null}, then the length of each row is assumed to be {@code sizePerOffset}. If + * {@code lengths} is {@code null}, then the length of each row is determined by adjacent elements in + * {@code offsets}. If both are non-{@code null}, then the length of each row is determined by {@code lengths}. + * + * @param ii the index to compute the size for + * @param sizePerOffset the size of each offset when fixed + * @param offsets the source IntChunk to determine the start location of each row + * @param lengths the source IntChunk to determine the length of each row + * @return the length of the given index + */ + @FinalDefault + default int computeSize( + final int ii, + final int sizePerOffset, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths) { + if (offsets == null) { + return sizePerOffset; + } + if (lengths == null) { + return offsets.get(ii + 1) - offsets.get(ii); + } + return lengths.get(ii); + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkReader.java similarity index 71% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkReader.java index 7b77b00911b..be811192228 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkReader.java @@ -7,51 +7,48 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; -public class FixedWidthChunkInputStreamGenerator { - private static final String DEBUG_NAME = "FixedWidthChunkInputStreamGenerator"; +public class FixedWidthChunkReader extends BaseChunkReader> { + private static final String DEBUG_NAME = "FixedWidthWriter"; @FunctionalInterface public interface TypeConversion { T apply(DataInput in) throws IOException; } - /** - * Generic input stream reading from arrow's buffer and convert directly to java type. - * - * If useDeephavenNulls is enabled, then the conversion method must properly return a null value. - * - * @param elementSize the number of bytes per element (element size is fixed) - * @param options the stream reader options - * @param conversion the conversion method from input stream to the result type - * @param fieldNodeIter arrow field node iterator - * @param bufferInfoIter arrow buffer info iterator - * @param outChunk the returned chunk from an earlier record batch - * @param outOffset the offset to start writing into {@code outChunk} - * @param totalRows the total known rows for this column; if known (else 0) - * @param is data input stream - * @param the result type - * @return the resulting chunk of the buffer that is read - */ - public static WritableObjectChunk extractChunkFromInputStreamWithTypeConversion( + private final boolean useDeephavenNulls; + private final int elementSize; + private final ChunkReader.Options options; + private final TypeConversion conversion; + + public FixedWidthChunkReader( final int elementSize, - final StreamReaderOptions options, - final TypeConversion conversion, - final Iterator fieldNodeIter, - final PrimitiveIterator.OfLong bufferInfoIter, - final DataInput is, - final WritableChunk outChunk, + final boolean dhNullable, + final ChunkReader.Options options, + final TypeConversion conversion) { + this.elementSize = elementSize; + this.options = options; + this.conversion = conversion; + this.useDeephavenNulls = dhNullable && options.useDeephavenNulls(); + } + + @Override + public WritableObjectChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -70,9 +67,6 @@ public static WritableObjectChunk extractChunkFromInputStreamWith final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; final long numValidityLongsPresent = Math.min(numValidityLongs, validityBuffer / 8); for (; jj < numValidityLongsPresent; ++jj) { @@ -93,7 +87,7 @@ public static WritableObjectChunk extractChunkFromInputStreamWith throw new IllegalStateException("payload buffer is too short for expected number of elements"); } - if (options.useDeephavenNulls()) { + if (useDeephavenNulls) { for (int ii = 0; ii < nodeInfo.numElements; ++ii) { chunk.set(outOffset + ii, conversion.apply(is)); } @@ -114,7 +108,7 @@ private static void useValidityBuffer( final int elementSize, final TypeConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableObjectChunk chunk, final int outOffset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkWriter.java new file mode 100644 index 00000000000..0301c516a2d --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkWriter.java @@ -0,0 +1,100 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataOutput; +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class FixedWidthChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "FixedWidthChunkWriter"; + + @FunctionalInterface + public interface Appender> { + void append(@NotNull DataOutput os, @NotNull SourceChunkType sourceValues, int offset) throws IOException; + } + + private final Appender appendItem; + + public FixedWidthChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + final int elementSize, + final boolean dhNullable, + final Appender appendItem) { + super(emptyChunkSupplier, elementSize, dhNullable); + this.appendItem = appendItem; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new FixedWidthChunkInputStream(context, subset, options); + } + + private class FixedWidthChunkInputStream extends BaseChunkInputStream> { + private FixedWidthChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final DataOutput dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // ensure we can cast all row keys to int + LongSizedDataStructure.intSize(DEBUG_NAME, subset.lastRowKey()); + + // write the payload buffer + subset.forAllRowKeys(rowKey -> { + try { + appendItem.append(dos, context.getChunk(), (int) rowKey); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java deleted file mode 100644 index edd8aaccb2a..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java +++ /dev/null @@ -1,161 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkInputStreamGenerator and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.primitive.function.ToFloatFunction; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.FloatChunk; -import io.deephaven.chunk.WritableFloatChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class FloatChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "FloatChunkInputStreamGenerator"; - - public static FloatChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static FloatChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToFloatFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableFloatChunk outChunk = WritableFloatChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsFloat(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new FloatChunkInputStreamGenerator(outChunk, Float.BYTES, rowOffset); - } - - FloatChunkInputStreamGenerator(final FloatChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new FloatChunkInputStream(options, subset); - } - - private class FloatChunkInputStream extends BaseChunkInputStream { - private FloatChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_FLOAT) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_FLOAT) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final float val = chunk.get((int) row); - dos.writeFloat(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("FloatChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkReader.java index df2bfa32071..a30b96fee24 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkReader.java @@ -1,83 +1,63 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkReader and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off package io.deephaven.extensions.barrage.chunk; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; +import io.deephaven.extensions.barrage.util.Float16; +import io.deephaven.util.QueryConstants; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.apache.arrow.flatbuf.Precision; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; -import java.util.function.Function; -import java.util.function.IntFunction; - -import static io.deephaven.util.QueryConstants.NULL_FLOAT; -public class FloatChunkReader implements ChunkReader { +public class FloatChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "FloatChunkReader"; - private final StreamReaderOptions options; - private final FloatConversion conversion; - - @FunctionalInterface - public interface FloatConversion { - float apply(float in); - FloatConversion IDENTITY = (float a) -> a; + public interface ToFloatTransformFunction> { + float get(WireChunkType wireValues, int wireOffset); } - public FloatChunkReader(StreamReaderOptions options) { - this(options, FloatConversion.IDENTITY); - } - - public FloatChunkReader(StreamReaderOptions options, FloatConversion conversion) { - this.options = options; - this.conversion = conversion; + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToFloatTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableFloatChunk::makeWritableChunk, + WritableChunk::asWritableFloatChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); } - public ChunkReader transform(Function transform) { - return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { - try (final WritableFloatChunk inner = FloatChunkReader.this.readChunk( - fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { + private final short precisionFlatBufId; + private final ChunkReader.Options options; - final WritableObjectChunk chunk = castOrCreateChunk( - outChunk, - Math.max(totalRows, inner.size()), - WritableObjectChunk::makeWritableChunk, - WritableChunk::asWritableObjectChunk); - - if (outChunk == null) { - // if we're not given an output chunk then we better be writing at the front of the new one - Assert.eqZero(outOffset, "outOffset"); - } - - for (int ii = 0; ii < inner.size(); ++ii) { - float value = inner.get(ii); - chunk.set(outOffset + ii, transform.apply(value)); - } - - return chunk; - } - }; + public FloatChunkReader( + final short precisionFlatbufId, + final ChunkReader.Options options) { + this.precisionFlatBufId = precisionFlatbufId; + this.options = options; } @Override - public WritableFloatChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableFloatChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -93,9 +73,6 @@ public WritableFloatChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -114,9 +91,9 @@ public WritableFloatChunk readChunk(Iterator readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( - final FloatConversion conversion, + final short precisionFlatBufId, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableFloatChunk chunk, final int offset) throws IOException { - if (conversion == FloatConversion.IDENTITY) { - for (int ii = 0; ii < nodeInfo.numElements; ++ii) { - chunk.set(offset + ii, is.readFloat()); - } - } else { - for (int ii = 0; ii < nodeInfo.numElements; ++ii) { - final float in = is.readFloat(); - final float out = in == NULL_FLOAT ? in : conversion.apply(in); - chunk.set(offset + ii, out); - } + switch (precisionFlatBufId) { + case Precision.HALF: + throw new IllegalStateException("Cannot use Deephaven nulls with half-precision floats"); + case Precision.SINGLE: + for (int ii = 0; ii < nodeInfo.numElements; ++ii) { + chunk.set(offset + ii, is.readFloat()); + } + break; + case Precision.DOUBLE: + for (int ii = 0; ii < nodeInfo.numElements; ++ii) { + final double v = is.readDouble(); + chunk.set(offset + ii, v == QueryConstants.NULL_DOUBLE ? QueryConstants.NULL_FLOAT : (float) v); + } + break; + default: + throw new IllegalStateException("Unsupported floating point precision: " + precisionFlatBufId); } } + @FunctionalInterface + private interface FloatSupplier { + float next() throws IOException; + } + + private static float floatCast(double a) { + return a == QueryConstants.NULL_DOUBLE ? QueryConstants.NULL_FLOAT : (float) a; + } + private static void useValidityBuffer( - final FloatConversion conversion, + final short precisionFlatBufId, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableFloatChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { @@ -173,18 +152,37 @@ private static void useValidityBuffer( int ei = 0; int pendingSkips = 0; + final int elementSize; + final FloatSupplier supplier; + switch (precisionFlatBufId) { + case Precision.HALF: + elementSize = Short.BYTES; + supplier = () -> Float16.toFloat(is.readShort()); + break; + case Precision.SINGLE: + elementSize = Float.BYTES; + supplier = is::readFloat; + break; + case Precision.DOUBLE: + elementSize = Double.BYTES; + supplier = () -> floatCast(is.readDouble()); + break; + default: + throw new IllegalStateException("Unsupported floating point precision: " + precisionFlatBufId); + } + for (int vi = 0; vi < numValidityWords; ++vi) { int bitsLeftInThisWord = Math.min(64, numElements - vi * 64); long validityWord = isValid.get(vi); do { if ((validityWord & 1) == 1) { if (pendingSkips > 0) { - is.skipBytes(pendingSkips * Float.BYTES); + is.skipBytes(pendingSkips * elementSize); chunk.fillWithNullValue(offset + ei, pendingSkips); ei += pendingSkips; pendingSkips = 0; } - chunk.set(offset + ei++, conversion.apply(is.readFloat())); + chunk.set(offset + ei++, supplier.next()); validityWord >>= 1; bitsLeftInThisWord--; } else { @@ -197,7 +195,7 @@ private static void useValidityBuffer( } if (pendingSkips > 0) { - is.skipBytes(pendingSkips * Float.BYTES); + is.skipBytes(pendingSkips * elementSize); chunk.fillWithNullValue(offset + ei, pendingSkips); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkWriter.java new file mode 100644 index 00000000000..02b27b8b882 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkWriter.java @@ -0,0 +1,101 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkWriter and run "./gradlew replicateBarrageUtils" to regenerate +// +// @formatter:off +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.FloatChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class FloatChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "FloatChunkWriter"; + public static final FloatChunkWriter> INSTANCE = new FloatChunkWriter<>( + FloatChunk::getEmptyChunk, FloatChunk::get); + + @FunctionalInterface + public interface ToFloatTransformFunction> { + float get(SourceChunkType sourceValues, int offset); + } + + private final ToFloatTransformFunction transform; + + public FloatChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToFloatTransformFunction transform) { + super(emptyChunkSupplier, Float.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new FloatChunkInputStream(context, subset, options); + } + + private class FloatChunkInputStream extends BaseChunkInputStream> { + private FloatChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeFloat(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java deleted file mode 100644 index 87bc61b8c6d..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java +++ /dev/null @@ -1,162 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkInputStreamGenerator and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off -package io.deephaven.extensions.barrage.chunk; - -import java.util.function.ToIntFunction; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class IntChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "IntChunkInputStreamGenerator"; - - public static IntChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static IntChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToIntFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableIntChunk outChunk = WritableIntChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsInt(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new IntChunkInputStreamGenerator(outChunk, Integer.BYTES, rowOffset); - } - - IntChunkInputStreamGenerator(final IntChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new IntChunkInputStream(options, subset); - } - - private class IntChunkInputStream extends BaseChunkInputStream { - private IntChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_INT) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_INT) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final int val = chunk.get((int) row); - dos.writeInt(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("IntChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkReader.java index edf333f054b..562bc6cd475 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkReader.java @@ -13,21 +13,38 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; import java.util.function.Function; -import java.util.function.IntFunction; import static io.deephaven.util.QueryConstants.NULL_INT; -public class IntChunkReader implements ChunkReader { +public class IntChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "IntChunkReader"; - private final StreamReaderOptions options; + + @FunctionalInterface + public interface ToIntTransformFunction> { + int get(WireChunkType wireValues, int wireOffset); + } + + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToIntTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableIntChunk::makeWritableChunk, + WritableChunk::asWritableIntChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); + } + + private final ChunkReader.Options options; private final IntConversion conversion; @FunctionalInterface @@ -37,16 +54,16 @@ public interface IntConversion { IntConversion IDENTITY = (int a) -> a; } - public IntChunkReader(StreamReaderOptions options) { + public IntChunkReader(ChunkReader.Options options) { this(options, IntConversion.IDENTITY); } - public IntChunkReader(StreamReaderOptions options, IntConversion conversion) { + public IntChunkReader(ChunkReader.Options options, IntConversion conversion) { this.options = options; this.conversion = conversion; } - public ChunkReader transform(Function transform) { + public ChunkReader> transform(Function transform) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { try (final WritableIntChunk inner = IntChunkReader.this.readChunk( fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { @@ -73,11 +90,15 @@ public ChunkReader transform(Function transform) { } @Override - public WritableIntChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableIntChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -93,9 +114,6 @@ public WritableIntChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -128,23 +146,10 @@ public WritableIntChunk readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( final IntConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableIntChunk chunk, final int offset) throws IOException { if (conversion == IntConversion.IDENTITY) { @@ -163,7 +168,7 @@ private static void useDeephavenNulls( private static void useValidityBuffer( final IntConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableIntChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkWriter.java new file mode 100644 index 00000000000..62bcbc864e0 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkWriter.java @@ -0,0 +1,101 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkWriter and run "./gradlew replicateBarrageUtils" to regenerate +// +// @formatter:off +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.IntChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class IntChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "IntChunkWriter"; + public static final IntChunkWriter> INSTANCE = new IntChunkWriter<>( + IntChunk::getEmptyChunk, IntChunk::get); + + @FunctionalInterface + public interface ToIntTransformFunction> { + int get(SourceChunkType sourceValues, int offset); + } + + private final ToIntTransformFunction transform; + + public IntChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToIntTransformFunction transform) { + super(emptyChunkSupplier, Integer.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new IntChunkInputStream(context, subset, options); + } + + private class IntChunkInputStream extends BaseChunkInputStream> { + private IntChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeInt(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkReader.java new file mode 100644 index 00000000000..144a15bbc49 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkReader.java @@ -0,0 +1,149 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.attributes.ChunkLengths; +import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataInput; +import java.io.IOException; +import java.util.Iterator; +import java.util.PrimitiveIterator; + +public class ListChunkReader extends BaseChunkReader> { + public enum Mode { + FIXED, DENSE, SPARSE + } + + private static final String DEBUG_NAME = "ListChunkReader"; + + private final Mode mode; + private final int fixedSizeLength; + private final ExpansionKernel kernel; + private final ChunkReader> componentReader; + + public ListChunkReader( + final Mode mode, + final int fixedSizeLength, + final ExpansionKernel kernel, + final ChunkReader> componentReader) { + this.mode = mode; + this.fixedSizeLength = fixedSizeLength; + this.componentReader = componentReader; + this.kernel = kernel; + } + + @Override + public WritableObjectChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + final long validityBufferLength = bufferInfoIter.nextLong(); + // have an offsets buffer if not every element is the same length + final long offsetsBufferLength = mode == Mode.FIXED ? 0 : bufferInfoIter.nextLong(); + // have a lengths buffer if ListView instead of List + final long lengthsBufferLength = mode != Mode.SPARSE ? 0 : bufferInfoIter.nextLong(); + + if (nodeInfo.numElements == 0) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, + validityBufferLength + offsetsBufferLength + lengthsBufferLength)); + try (final WritableChunk ignored = + componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { + return WritableObjectChunk.makeWritableChunk(nodeInfo.numElements); + } + } + + final WritableObjectChunk chunk; + final int numValidityLongs = (nodeInfo.numElements + 63) / 64; + final int numOffsets = nodeInfo.numElements + (mode == Mode.DENSE ? 1 : 0); + try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs); + final WritableIntChunk offsets = mode == Mode.FIXED + ? null + : WritableIntChunk.makeWritableChunk(numOffsets); + final WritableIntChunk lengths = mode != Mode.SPARSE + ? null + : WritableIntChunk.makeWritableChunk(nodeInfo.numElements)) { + + // Read validity buffer: + int jj = 0; + for (; jj < Math.min(numValidityLongs, validityBufferLength / 8); ++jj) { + isValid.set(jj, is.readLong()); + } + final long valBufRead = jj * 8L; + if (valBufRead < validityBufferLength) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, validityBufferLength - valBufRead)); + } + // we support short validity buffers + for (; jj < numValidityLongs; ++jj) { + isValid.set(jj, -1); // -1 is bit-wise representation of all ones + } + + // Read offsets: + if (offsets != null) { + final long offBufRead = (long) numOffsets * Integer.BYTES; + if (offsetsBufferLength < offBufRead) { + throw new IllegalStateException( + "list offset buffer is too short for the expected number of elements"); + } + for (int ii = 0; ii < numOffsets; ++ii) { + offsets.set(ii, is.readInt()); + } + if (offBufRead < offsetsBufferLength) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, offsetsBufferLength - offBufRead)); + } + } + + // Read lengths: + if (lengths != null) { + final long lenBufRead = ((long) nodeInfo.numElements) * Integer.BYTES; + if (lengthsBufferLength < lenBufRead) { + throw new IllegalStateException( + "list sizes buffer is too short for the expected number of elements"); + } + for (int ii = 0; ii < nodeInfo.numElements; ++ii) { + lengths.set(ii, is.readInt()); + } + if (lenBufRead < lengthsBufferLength) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, lengthsBufferLength - lenBufRead)); + } + } + + try (final WritableChunk inner = + componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { + // noinspection unchecked + chunk = (WritableObjectChunk) kernel.contract(inner, fixedSizeLength, offsets, lengths, + outChunk, outOffset, totalRows); + + long nextValid = 0; + for (int ii = 0; ii < nodeInfo.numElements;) { + if ((ii % 64) == 0) { + nextValid = ~isValid.get(ii / 64); + } + if ((nextValid & 0x1) == 0x1) { + chunk.set(outOffset + ii, null); + } + final int numToSkip = Math.min( + Long.numberOfTrailingZeros(nextValid & (~0x1)), + 64 - (ii % 64)); + nextValid >>= numToSkip; + ii += numToSkip; + } + } + } + + return chunk; + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkWriter.java new file mode 100644 index 00000000000..4bbba35be08 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ListChunkWriter.java @@ -0,0 +1,247 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetBuilderSequential; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; + +public class ListChunkWriter> + extends BaseChunkWriter> { + private static final String DEBUG_NAME = "ListChunkWriter"; + + private final ListChunkReader.Mode mode; + private final int fixedSizeLength; + private final ExpansionKernel kernel; + private final ChunkWriter componentWriter; + + public ListChunkWriter( + final ListChunkReader.Mode mode, + final int fixedSizeLength, + final ExpansionKernel kernel, + final ChunkWriter componentWriter) { + super(ObjectChunk::getEmptyChunk, 0, false); + this.mode = mode; + this.fixedSizeLength = fixedSizeLength; + this.kernel = kernel; + this.componentWriter = componentWriter; + } + + @Override + public Context makeContext( + @NotNull final ObjectChunk chunk, + final long rowOffset) { + return new Context(chunk, rowOffset); + } + + public final class Context extends ChunkWriter.Context> { + private final WritableIntChunk offsets; + private final ChunkWriter.Context innerContext; + + public Context( + @NotNull final ObjectChunk chunk, + final long rowOffset) { + super(chunk, rowOffset); + + if (mode == ListChunkReader.Mode.FIXED) { + offsets = null; + } else { + int numOffsets = chunk.size() + (mode == ListChunkReader.Mode.DENSE ? 1 : 0); + offsets = WritableIntChunk.makeWritableChunk(numOffsets); + } + + // noinspection unchecked + innerContext = componentWriter.makeContext( + (ComponentChunkType) kernel.expand(chunk, fixedSizeLength, offsets), 0); + } + + @Override + public void close() { + super.close(); + offsets.close(); + innerContext.close(); + } + } + + @Override + public DrainableColumn getInputStream( + @NotNull final ChunkWriter.Context> context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new ListChunkInputStream((Context) context, subset, options); + } + + private class ListChunkInputStream extends BaseChunkInputStream { + + private int cachedSize = -1; + private final WritableIntChunk myOffsets; + private final DrainableColumn innerColumn; + + private ListChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet mySubset, + @NotNull final ChunkReader.Options options) throws IOException { + super(context, mySubset, options); + + if (subset == null || subset.size() == context.size()) { + // we are writing everything + myOffsets = null; + innerColumn = componentWriter.getInputStream(context.innerContext, null, options); + } else { + if (fixedSizeLength != 0) { + myOffsets = null; + } else { + // note that we maintain dense offsets within the writer, but write per the wire format + myOffsets = WritableIntChunk.makeWritableChunk(context.size() + 1); + myOffsets.setSize(0); + myOffsets.add(0); + } + + final RowSetBuilderSequential innerSubsetBuilder = RowSetFactory.builderSequential(); + subset.forAllRowKeys(key -> { + final int startOffset = context.offsets.get(LongSizedDataStructure.intSize(DEBUG_NAME, key)); + final int endOffset = context.offsets.get(LongSizedDataStructure.intSize(DEBUG_NAME, key + 1)); + if (fixedSizeLength == 0) { + myOffsets.add(endOffset - startOffset + myOffsets.get(myOffsets.size() - 1)); + } + if (endOffset > startOffset) { + innerSubsetBuilder.appendRange(startOffset, endOffset - 1); + } + }); + try (final RowSet innerSubset = innerSubsetBuilder.build()) { + innerColumn = componentWriter.getInputStream(context.innerContext, innerSubset, options); + } + } + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + innerColumn.visitFieldNodes(listener); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + final int numElements = subset.intSize(DEBUG_NAME); + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(numElements) : 0); + + // offsets + if (mode != ListChunkReader.Mode.FIXED) { + long numOffsetBytes = Integer.BYTES * ((long) numElements); + if (numElements > 0 && mode == ListChunkReader.Mode.DENSE) { + // we need an extra offset for the end of the last element + numOffsetBytes += Integer.BYTES; + } + listener.noteLogicalBuffer(padBufferSize(numOffsetBytes)); + } + + // lengths + if (mode == ListChunkReader.Mode.SPARSE) { + long numLengthsBytes = Integer.BYTES * ((long) numElements); + listener.noteLogicalBuffer(padBufferSize(numLengthsBytes)); + } + + // payload + innerColumn.visitBuffers(listener); + } + + @Override + public void close() throws IOException { + super.close(); + if (myOffsets != null) { + myOffsets.close(); + } + innerColumn.close(); + } + + @Override + protected int getRawSize() throws IOException { + if (cachedSize == -1) { + long size; + + // validity + final int numElements = subset.intSize(DEBUG_NAME); + size = sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)) : 0; + + // offsets + if (mode != ListChunkReader.Mode.FIXED) { + long numOffsetBytes = Integer.BYTES * ((long) numElements); + if (numElements > 0 && mode == ListChunkReader.Mode.DENSE) { + // we need an extra offset for the end of the last element + numOffsetBytes += Integer.BYTES; + } + size += padBufferSize(numOffsetBytes); + } + + // lengths + if (mode == ListChunkReader.Mode.SPARSE) { + long numLengthsBytes = Integer.BYTES * ((long) numElements); + size += padBufferSize(numLengthsBytes); + } + + size += innerColumn.available(); + cachedSize = LongSizedDataStructure.intSize(DEBUG_NAME, size); + } + + return cachedSize; + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + read = true; + long bytesWritten = 0; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + // write the validity array with LSB indexing + bytesWritten += writeValidityBuffer(dos); + + // write offsets array + if (mode == ListChunkReader.Mode.DENSE) { + // write down only offset (+1) buffer + final WritableIntChunk offsetsToUse = myOffsets == null ? context.offsets : myOffsets; + for (int i = 0; i < offsetsToUse.size(); ++i) { + dos.writeInt(offsetsToUse.get(i)); + } + bytesWritten += ((long) offsetsToUse.size()) * Integer.BYTES; + bytesWritten += writePadBuffer(dos, bytesWritten); + } else if (mode == ListChunkReader.Mode.SPARSE) { + // write down offset buffer + final WritableIntChunk offsetsToUse = myOffsets == null ? context.offsets : myOffsets; + + // note that we have one extra offset because we keep dense offsets internally + for (int i = 0; i < offsetsToUse.size() - 1; ++i) { + dos.writeInt(offsetsToUse.get(i)); + } + bytesWritten += ((long) offsetsToUse.size() - 1) * Integer.BYTES; + bytesWritten += writePadBuffer(dos, bytesWritten); + + // write down length buffer + for (int i = 0; i < offsetsToUse.size() - 1; ++i) { + dos.writeInt(offsetsToUse.get(i + 1) - offsetsToUse.get(i)); + } + bytesWritten += ((long) offsetsToUse.size() - 1) * Integer.BYTES; + bytesWritten += writePadBuffer(dos, bytesWritten); + } // the other mode is fixed, which doesn't have an offset or length buffer + + bytesWritten += innerColumn.drainTo(outputStream); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java deleted file mode 100644 index 671d972ccce..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java +++ /dev/null @@ -1,162 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkInputStreamGenerator and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off -package io.deephaven.extensions.barrage.chunk; - -import java.util.function.ToLongFunction; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class LongChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "LongChunkInputStreamGenerator"; - - public static LongChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static LongChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToLongFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableLongChunk outChunk = WritableLongChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsLong(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new LongChunkInputStreamGenerator(outChunk, Long.BYTES, rowOffset); - } - - LongChunkInputStreamGenerator(final LongChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new LongChunkInputStream(options, subset); - } - - private class LongChunkInputStream extends BaseChunkInputStream { - private LongChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_LONG) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_LONG) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final long val = chunk.get((int) row); - dos.writeLong(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("LongChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkReader.java index e96385b6740..decad1d77fe 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkReader.java @@ -13,21 +13,38 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; import java.util.function.Function; -import java.util.function.IntFunction; import static io.deephaven.util.QueryConstants.NULL_LONG; -public class LongChunkReader implements ChunkReader { +public class LongChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "LongChunkReader"; - private final StreamReaderOptions options; + + @FunctionalInterface + public interface ToLongTransformFunction> { + long get(WireChunkType wireValues, int wireOffset); + } + + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToLongTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableLongChunk::makeWritableChunk, + WritableChunk::asWritableLongChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); + } + + private final ChunkReader.Options options; private final LongConversion conversion; @FunctionalInterface @@ -37,16 +54,16 @@ public interface LongConversion { LongConversion IDENTITY = (long a) -> a; } - public LongChunkReader(StreamReaderOptions options) { + public LongChunkReader(ChunkReader.Options options) { this(options, LongConversion.IDENTITY); } - public LongChunkReader(StreamReaderOptions options, LongConversion conversion) { + public LongChunkReader(ChunkReader.Options options, LongConversion conversion) { this.options = options; this.conversion = conversion; } - public ChunkReader transform(Function transform) { + public ChunkReader> transform(Function transform) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { try (final WritableLongChunk inner = LongChunkReader.this.readChunk( fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { @@ -73,11 +90,15 @@ public ChunkReader transform(Function transform) { } @Override - public WritableLongChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableLongChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -93,9 +114,6 @@ public WritableLongChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -128,23 +146,10 @@ public WritableLongChunk readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( final LongConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableLongChunk chunk, final int offset) throws IOException { if (conversion == LongConversion.IDENTITY) { @@ -163,7 +168,7 @@ private static void useDeephavenNulls( private static void useValidityBuffer( final LongConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableLongChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkWriter.java new file mode 100644 index 00000000000..b9574744fd9 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkWriter.java @@ -0,0 +1,101 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkWriter and run "./gradlew replicateBarrageUtils" to regenerate +// +// @formatter:off +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.LongChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class LongChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "LongChunkWriter"; + public static final LongChunkWriter> INSTANCE = new LongChunkWriter<>( + LongChunk::getEmptyChunk, LongChunk::get); + + @FunctionalInterface + public interface ToLongTransformFunction> { + long get(SourceChunkType sourceValues, int offset); + } + + private final ToLongTransformFunction transform; + + public LongChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToLongTransformFunction transform) { + super(emptyChunkSupplier, Long.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new LongChunkInputStream(context, subset, options); + } + + private class LongChunkInputStream extends BaseChunkInputStream> { + private LongChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeLong(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkReader.java new file mode 100644 index 00000000000..a3b45dc837b --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkReader.java @@ -0,0 +1,47 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataInput; +import java.io.IOException; +import java.util.Iterator; +import java.util.PrimitiveIterator; + +public class NullChunkReader> extends BaseChunkReader { + + private final ChunkType resultType; + + public NullChunkReader(Class destType) { + this.resultType = getChunkTypeFor(destType); + } + + @Override + public ReadChunkType readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + // null nodes have no buffers + + final WritableChunk chunk = castOrCreateChunk( + outChunk, + Math.max(totalRows, nodeInfo.numElements), + resultType::makeWritableChunk, + c -> c); + + chunk.fillWithNullValue(0, nodeInfo.numElements); + + // noinspection unchecked + return (ReadChunkType) chunk; + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkWriter.java new file mode 100644 index 00000000000..43a2c07869f --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/NullChunkWriter.java @@ -0,0 +1,52 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; + +public class NullChunkWriter> extends BaseChunkWriter { + + public NullChunkWriter() { + super(() -> null, 0, true); + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context chunk, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new NullDrainableColumn(); + } + + public static class NullDrainableColumn extends DrainableColumn { + + @Override + public void visitFieldNodes(FieldNodeListener listener) {} + + @Override + public void visitBuffers(BufferListener listener) {} + + @Override + public int nullCount() { + return 0; + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + return 0; + } + + @Override + public int available() throws IOException { + return 0; + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java deleted file mode 100644 index 4fd81b47d03..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java +++ /dev/null @@ -1,161 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY -// ****** Edit CharChunkInputStreamGenerator and run "./gradlew replicateBarrageUtils" to regenerate -// -// @formatter:off -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.util.pools.PoolableChunk; -import io.deephaven.engine.primitive.function.ToShortFunction; -import io.deephaven.engine.rowset.RowSet; -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.ShortChunk; -import io.deephaven.chunk.WritableShortChunk; -import io.deephaven.util.type.TypeUtils; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -import static io.deephaven.util.QueryConstants.*; - -public class ShortChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "ShortChunkInputStreamGenerator"; - - public static ShortChunkInputStreamGenerator convertBoxed( - final ObjectChunk inChunk, final long rowOffset) { - return convertWithTransform(inChunk, rowOffset, TypeUtils::unbox); - } - - public static ShortChunkInputStreamGenerator convertWithTransform( - final ObjectChunk inChunk, final long rowOffset, final ToShortFunction transform) { - // This code path is utilized for arrays and vectors of DateTimes, LocalDate, and LocalTime, which cannot be - // reinterpreted. - WritableShortChunk outChunk = WritableShortChunk.makeWritableChunk(inChunk.size()); - for (int i = 0; i < inChunk.size(); ++i) { - T value = inChunk.get(i); - outChunk.set(i, transform.applyAsShort(value)); - } - // inChunk is a transfer of ownership to us, but we've converted what we need, so we must close it now - if (inChunk instanceof PoolableChunk) { - ((PoolableChunk) inChunk).close(); - } - return new ShortChunkInputStreamGenerator(outChunk, Short.BYTES, rowOffset); - } - - ShortChunkInputStreamGenerator(final ShortChunk chunk, final int elementSize, final long rowOffset) { - super(chunk, elementSize, rowOffset); - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) { - return new ShortChunkInputStream(options, subset); - } - - private class ShortChunkInputStream extends BaseChunkInputStream { - private ShortChunkInputStream(final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (options.useDeephavenNulls()) { - return 0; - } - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) == NULL_SHORT) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); - // payload - long length = elementSize * subset.size(); - final long bytesExtended = length & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - length += 8 - bytesExtended; - } - listener.noteLogicalBuffer(length); - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - long bytesWritten = 0; - read = true; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(row -> { - if (chunk.get((int) row) != NULL_SHORT) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize()); - } - - // write the included values - subset.forAllRowKeys(row -> { - try { - final short val = chunk.get((int) row); - dos.writeShort(val); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - }); - - bytesWritten += elementSize * subset.size(); - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize("ShortChunkInputStreamGenerator", bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkReader.java index 1bd92351d6c..b90ce6b6928 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkReader.java @@ -13,21 +13,38 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.DataInput; import java.io.IOException; import java.util.Iterator; import java.util.PrimitiveIterator; import java.util.function.Function; -import java.util.function.IntFunction; import static io.deephaven.util.QueryConstants.NULL_SHORT; -public class ShortChunkReader implements ChunkReader { +public class ShortChunkReader extends BaseChunkReader> { private static final String DEBUG_NAME = "ShortChunkReader"; - private final StreamReaderOptions options; + + @FunctionalInterface + public interface ToShortTransformFunction> { + short get(WireChunkType wireValues, int wireOffset); + } + + public static , T extends ChunkReader> ChunkReader> transformTo( + final T wireReader, + final ToShortTransformFunction wireTransform) { + return new TransformingChunkReader<>( + wireReader, + WritableShortChunk::makeWritableChunk, + WritableChunk::asWritableShortChunk, + (wireValues, outChunk, wireOffset, outOffset) -> outChunk.set( + outOffset, wireTransform.get(wireValues, wireOffset))); + } + + private final ChunkReader.Options options; private final ShortConversion conversion; @FunctionalInterface @@ -37,16 +54,16 @@ public interface ShortConversion { ShortConversion IDENTITY = (short a) -> a; } - public ShortChunkReader(StreamReaderOptions options) { + public ShortChunkReader(ChunkReader.Options options) { this(options, ShortConversion.IDENTITY); } - public ShortChunkReader(StreamReaderOptions options, ShortConversion conversion) { + public ShortChunkReader(ChunkReader.Options options, ShortConversion conversion) { this.options = options; this.conversion = conversion; } - public ChunkReader transform(Function transform) { + public ChunkReader> transform(Function transform) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, totalRows) -> { try (final WritableShortChunk inner = ShortChunkReader.this.readChunk( fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { @@ -73,11 +90,15 @@ public ChunkReader transform(Function transform) { } @Override - public WritableShortChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + public WritableShortChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); @@ -93,9 +114,6 @@ public WritableShortChunk readChunk(Iterator isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - if (options.useDeephavenNulls() && validityBuffer != 0) { - throw new IllegalStateException("validity buffer is non-empty, but is unnecessary"); - } int jj = 0; for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { isValid.set(jj, is.readLong()); @@ -128,23 +146,10 @@ public WritableShortChunk readChunk(Iterator> T castOrCreateChunk( - final WritableChunk outChunk, - final int numRows, - final IntFunction chunkFactory, - final Function, T> castFunction) { - if (outChunk != null) { - return castFunction.apply(outChunk); - } - final T newChunk = chunkFactory.apply(numRows); - newChunk.setSize(numRows); - return newChunk; - } - private static void useDeephavenNulls( final ShortConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableShortChunk chunk, final int offset) throws IOException { if (conversion == ShortConversion.IDENTITY) { @@ -163,7 +168,7 @@ private static void useDeephavenNulls( private static void useValidityBuffer( final ShortConversion conversion, final DataInput is, - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo, + final ChunkWriter.FieldNodeInfo nodeInfo, final WritableShortChunk chunk, final int offset, final WritableLongChunk isValid) throws IOException { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkWriter.java new file mode 100644 index 00000000000..23f0b5f3149 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkWriter.java @@ -0,0 +1,101 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkWriter and run "./gradlew replicateBarrageUtils" to regenerate +// +// @formatter:off +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSet; +import com.google.common.io.LittleEndianDataOutputStream; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.chunk.ShortChunk; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.function.Supplier; + +public class ShortChunkWriter> extends BaseChunkWriter { + private static final String DEBUG_NAME = "ShortChunkWriter"; + public static final ShortChunkWriter> INSTANCE = new ShortChunkWriter<>( + ShortChunk::getEmptyChunk, ShortChunk::get); + + @FunctionalInterface + public interface ToShortTransformFunction> { + short get(SourceChunkType sourceValues, int offset); + } + + private final ToShortTransformFunction transform; + + public ShortChunkWriter( + @NotNull final Supplier emptyChunkSupplier, + @Nullable final ToShortTransformFunction transform) { + super(emptyChunkSupplier, Short.BYTES, true); + this.transform = transform; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new ShortChunkInputStream(context, subset, options); + } + + private class ShortChunkInputStream extends BaseChunkInputStream> { + private ShortChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(final FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize()) : 0); + // payload + long length = elementSize * subset.size(); + listener.noteLogicalBuffer(padBufferSize(length)); + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + long bytesWritten = 0; + read = true; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write the payload buffer + subset.forAllRowKeys(row -> { + try { + dos.writeShort(transform.get(context.getChunk(), (int) row)); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + + bytesWritten += elementSize * subset.size(); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java similarity index 76% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderInputStreamGenerator.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java index f2a5cdc552d..4387644e361 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java @@ -4,9 +4,9 @@ package io.deephaven.extensions.barrage.chunk; import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.BufferListener; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.DrainableColumn; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator.FieldNodeListener; +import io.deephaven.extensions.barrage.chunk.ChunkWriter.BufferListener; +import io.deephaven.extensions.barrage.chunk.ChunkWriter.DrainableColumn; +import io.deephaven.extensions.barrage.chunk.ChunkWriter.FieldNodeListener; import java.io.IOException; import java.io.OutputStream; @@ -14,11 +14,11 @@ /** * This helper class is used to generate only the header of an arrow list that contains a single element. */ -public class SingleElementListHeaderInputStreamGenerator extends DrainableColumn { +public class SingleElementListHeaderWriter extends DrainableColumn { private final int numElements; - public SingleElementListHeaderInputStreamGenerator(final int numElements) { + public SingleElementListHeaderWriter(final int numElements) { this.numElements = numElements; } @@ -41,7 +41,6 @@ public int nullCount() { return 0; } - @SuppressWarnings("UnstableApiUsage") @Override public int drainTo(final OutputStream outputStream) throws IOException { // allow this input stream to be re-read diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/TransformingChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/TransformingChunkReader.java new file mode 100644 index 00000000000..931894da676 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/TransformingChunkReader.java @@ -0,0 +1,69 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataInput; +import java.io.IOException; +import java.util.Iterator; +import java.util.PrimitiveIterator; +import java.util.function.Function; +import java.util.function.IntFunction; + +/** + * A {@link ChunkReader} that reads a chunk of wire values and transforms them into a different chunk type. + * + * @param the input chunk type + * @param the output chunk type + */ +public class TransformingChunkReader, OutputChunkType extends WritableChunk> + extends BaseChunkReader { + + public interface TransformFunction, OutputChunkType extends WritableChunk> { + void apply(InputChunkType wireValues, OutputChunkType outChunk, int wireOffset, int outOffset); + } + + private final ChunkReader wireChunkReader; + private final IntFunction chunkFactory; + private final Function, OutputChunkType> castFunction; + private final TransformFunction transformFunction; + + public TransformingChunkReader( + @NotNull final ChunkReader wireChunkReader, + final IntFunction chunkFactory, + final Function, OutputChunkType> castFunction, + final TransformFunction transformFunction) { + this.wireChunkReader = wireChunkReader; + this.chunkFactory = chunkFactory; + this.castFunction = castFunction; + this.transformFunction = transformFunction; + } + + @Override + public OutputChunkType readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + try (final InputChunkType wireValues = wireChunkReader.readChunk(fieldNodeIter, bufferInfoIter, is)) { + final OutputChunkType chunk = castOrCreateChunk( + outChunk, Math.max(totalRows, wireValues.size()), chunkFactory, castFunction); + if (outChunk == null) { + // if we're not given an output chunk then we better be writing at the front of the new one + Assert.eqZero(outOffset, "outOffset"); + } + for (int ii = 0; ii < wireValues.size(); ++ii) { + transformFunction.apply(wireValues, chunk, ii, outOffset + ii); + } + return chunk; + } + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkReader.java new file mode 100644 index 00000000000..77d01fed6f2 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkReader.java @@ -0,0 +1,136 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableIntChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.WritableObjectChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataInput; +import java.io.IOException; +import java.util.Iterator; +import java.util.PrimitiveIterator; + +public class VarBinaryChunkReader implements ChunkReader> { + private static final String DEBUG_NAME = "VarBinaryChunkReader"; + + public interface Mapper { + T constructFrom(byte[] buf, int offset, int length) throws IOException; + } + + private final Mapper mapper; + + public VarBinaryChunkReader(final Mapper mapper) { + this.mapper = mapper; + } + + @Override + public WritableObjectChunk readChunk( + @NotNull final Iterator fieldNodeIter, + @NotNull final PrimitiveIterator.OfLong bufferInfoIter, + @NotNull final DataInput is, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) throws IOException { + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long offsetsBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); + + final int numElements = nodeInfo.numElements; + final WritableObjectChunk chunk; + if (outChunk != null) { + chunk = outChunk.asWritableObjectChunk(); + } else { + final int numRows = Math.max(totalRows, numElements); + chunk = WritableObjectChunk.makeWritableChunk(numRows); + chunk.setSize(numRows); + } + + if (numElements == 0) { + return chunk; + } + + final int numValidityWords = (numElements + 63) / 64; + try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityWords); + final WritableIntChunk offsets = WritableIntChunk.makeWritableChunk(numElements + 1)) { + // Read validity buffer: + int jj = 0; + for (; jj < Math.min(numValidityWords, validityBuffer / 8); ++jj) { + isValid.set(jj, is.readLong()); + } + final long valBufRead = jj * 8L; + if (valBufRead < validityBuffer) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, validityBuffer - valBufRead)); + } + // we support short validity buffers + for (; jj < numValidityWords; ++jj) { + isValid.set(jj, -1); // -1 is bit-wise representation of all ones + } + + // Read offsets: + final long offBufRead = (numElements + 1L) * Integer.BYTES; + if (offsetsBuffer < offBufRead) { + throw new IllegalStateException("offset buffer is too short for the expected number of elements"); + } + for (int i = 0; i < numElements + 1; ++i) { + offsets.set(i, is.readInt()); + } + if (offBufRead < offsetsBuffer) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, offsetsBuffer - offBufRead)); + } + + // Read data: + final int bytesRead = LongSizedDataStructure.intSize(DEBUG_NAME, payloadBuffer); + final byte[] serializedData = new byte[bytesRead]; + is.readFully(serializedData); + + // Deserialize: + int ei = 0; + int pendingSkips = 0; + + for (int vi = 0; vi < numValidityWords; ++vi) { + int bitsLeftInThisWord = Math.min(64, numElements - vi * 64); + long validityWord = isValid.get(vi); + do { + if ((validityWord & 1) == 1) { + if (pendingSkips > 0) { + chunk.fillWithNullValue(outOffset + ei, pendingSkips); + ei += pendingSkips; + pendingSkips = 0; + } + final int offset = offsets.get(ei); + final int length = offsets.get(ei + 1) - offset; + Assert.geq(length, "length", 0); + if (offset + length > serializedData.length) { + throw new IllegalStateException("not enough data was serialized to parse this element: " + + "elementIndex=" + ei + " offset=" + offset + " length=" + length + + " serializedLen=" + serializedData.length); + } + chunk.set(outOffset + ei++, mapper.constructFrom(serializedData, offset, length)); + validityWord >>= 1; + bitsLeftInThisWord--; + } else { + final int skips = Math.min(Long.numberOfTrailingZeros(validityWord), bitsLeftInThisWord); + pendingSkips += skips; + validityWord >>= skips; + bitsLeftInThisWord -= skips; + } + } while (bitsLeftInThisWord > 0); + } + + if (pendingSkips > 0) { + chunk.fillWithNullValue(outOffset + ei, pendingSkips); + } + } + + return chunk; + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java similarity index 51% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java index b6c85018fb6..aa15f6b2493 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java @@ -5,12 +5,10 @@ import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; -import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.util.pools.ChunkPoolConstants; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.engine.rowset.RowSet; @@ -19,19 +17,176 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import java.io.DataInput; import java.io.IOException; import java.io.OutputStream; import java.util.ArrayList; -import java.util.Iterator; -import java.util.PrimitiveIterator; -public class VarBinaryChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { +public class VarBinaryChunkWriter extends BaseChunkWriter> { private static final String DEBUG_NAME = "ObjectChunkInputStream Serialization"; private static final int BYTE_CHUNK_SIZE = ChunkPoolConstants.LARGEST_POOLED_CHUNK_CAPACITY; + public interface Appender { + void append(OutputStream out, T item) throws IOException; + } + private final Appender appendItem; + public VarBinaryChunkWriter( + final Appender appendItem) { + super(ObjectChunk::getEmptyChunk, 0, false); + this.appendItem = appendItem; + } + + @Override + public DrainableColumn getInputStream( + @NotNull final ChunkWriter.Context> context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + return new ObjectChunkInputStream((Context) context, subset, options); + } + + @Override + public Context makeContext( + @NotNull final ObjectChunk chunk, + final long rowOffset) { + return new Context(chunk, rowOffset); + } + + public final class Context extends ChunkWriter.Context> { + private final ByteStorage byteStorage; + + public Context( + @NotNull final ObjectChunk chunk, + final long rowOffset) { + super(chunk, rowOffset); + + byteStorage = new ByteStorage(chunk.size() == 0 ? 0 : (chunk.size() + 1)); + + if (chunk.size() > 0) { + byteStorage.offsets.set(0, 0); + } + + for (int ii = 0; ii < chunk.size(); ++ii) { + if (chunk.isNullAt(ii)) { + continue; + } + try { + appendItem.append(byteStorage, chunk.get(ii)); + } catch (final IOException ioe) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", ioe); + } + byteStorage.offsets.set(ii + 1, byteStorage.size()); + } + } + } + + private class ObjectChunkInputStream extends BaseChunkInputStream { + + private int cachedSize = -1; + + private ObjectChunkInputStream( + @NotNull final Context context, + @Nullable final RowSet subset, + @NotNull final ChunkReader.Options options) throws IOException { + super(context, subset, options); + } + + @Override + public void visitFieldNodes(FieldNodeListener listener) { + listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); + } + + @Override + public void visitBuffers(final BufferListener listener) { + // validity + final int numElements = subset.intSize(DEBUG_NAME); + listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(numElements) : 0); + + // offsets + long numOffsetBytes = Integer.BYTES * (((long) numElements) + (numElements > 0 ? 1 : 0)); + listener.noteLogicalBuffer(padBufferSize(numOffsetBytes)); + + // payload + final MutableLong numPayloadBytes = new MutableLong(); + subset.forAllRowKeyRanges((s, e) -> { + numPayloadBytes.add(context.byteStorage.getPayloadSize((int) s, (int) e)); + }); + listener.noteLogicalBuffer(padBufferSize(numPayloadBytes.get())); + } + + @Override + protected int getRawSize() { + if (cachedSize == -1) { + MutableLong totalCachedSize = new MutableLong(0L); + if (sendValidityBuffer()) { + totalCachedSize.add(getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME))); + } + + // there are n+1 offsets; it is not assumed first offset is zero + if (!subset.isEmpty() && subset.size() == context.byteStorage.offsets.size() - 1) { + totalCachedSize.add(context.byteStorage.offsets.size() * (long) Integer.BYTES); + totalCachedSize.add(context.byteStorage.size()); + } else { + totalCachedSize.add(subset.isEmpty() ? 0 : Integer.BYTES); // account for the n+1 offset + subset.forAllRowKeyRanges((s, e) -> { + // account for offsets + totalCachedSize.add((e - s + 1) * Integer.BYTES); + + // account for payload + totalCachedSize.add(context.byteStorage.getPayloadSize((int) s, (int) e)); + }); + } + + if (!subset.isEmpty() && (subset.size() & 0x1) == 0) { + // then we must also align offset array + totalCachedSize.add(Integer.BYTES); + } + cachedSize = LongSizedDataStructure.intSize(DEBUG_NAME, totalCachedSize.get()); + } + return cachedSize; + } + + @Override + public int drainTo(final OutputStream outputStream) throws IOException { + if (read || subset.isEmpty()) { + return 0; + } + + read = true; + long bytesWritten = 0; + final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); + + // write the validity buffer + bytesWritten += writeValidityBuffer(dos); + + // write offsets array + dos.writeInt(0); + + final MutableInt logicalSize = new MutableInt(); + subset.forAllRowKeys((idx) -> { + try { + logicalSize.add(LongSizedDataStructure.intSize("int cast", + context.byteStorage.getPayloadSize((int) idx, (int) idx))); + dos.writeInt(logicalSize.get()); + } catch (final IOException e) { + throw new UncheckedDeephavenException("couldn't drain data to OutputStream", e); + } + }); + bytesWritten += Integer.BYTES * (subset.size() + 1); + + if ((subset.size() & 0x1) == 0) { + // then we must pad to align next buffer + dos.writeInt(0); + bytesWritten += Integer.BYTES; + } + + bytesWritten += context.byteStorage.writePayload(dos, 0, subset.intSize() - 1); + bytesWritten += writePadBuffer(dos, bytesWritten); + return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); + } + } + public static class ByteStorage extends OutputStream implements SafeCloseable { private final WritableLongChunk offsets; @@ -177,320 +332,4 @@ public void close() { } } } - - private ByteStorage byteStorage = null; - - public interface Appender { - void append(OutputStream out, T item) throws IOException; - } - - public interface Mapper { - T constructFrom(byte[] buf, int offset, int length) throws IOException; - } - - VarBinaryChunkInputStreamGenerator(final ObjectChunk chunk, - final long rowOffset, - final Appender appendItem) { - super(chunk, 0, rowOffset); - this.appendItem = appendItem; - } - - private synchronized void computePayload() throws IOException { - if (byteStorage != null) { - return; - } - byteStorage = new ByteStorage(chunk.size() == 0 ? 0 : (chunk.size() + 1)); - - if (chunk.size() > 0) { - byteStorage.offsets.set(0, 0); - } - for (int i = 0; i < chunk.size(); ++i) { - if (chunk.get(i) != null) { - appendItem.append(byteStorage, chunk.get(i)); - } - byteStorage.offsets.set(i + 1, byteStorage.size()); - } - } - - @Override - protected void onReferenceCountAtZero() { - super.onReferenceCountAtZero(); - if (byteStorage != null) { - byteStorage.close(); - } - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, @Nullable final RowSet subset) - throws IOException { - computePayload(); - return new ObjectChunkInputStream(options, subset); - } - - private class ObjectChunkInputStream extends BaseChunkInputStream { - - private int cachedSize = -1; - - private ObjectChunkInputStream( - final StreamReaderOptions options, final RowSet subset) { - super(chunk, options, subset); - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(i -> { - if (chunk.get((int) i) == null) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - final int numElements = subset.intSize(DEBUG_NAME); - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(numElements) : 0); - - // offsets - long numOffsetBytes = Integer.BYTES * (((long) numElements) + (numElements > 0 ? 1 : 0)); - final long bytesExtended = numOffsetBytes & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - numOffsetBytes += 8 - bytesExtended; - } - listener.noteLogicalBuffer(numOffsetBytes); - - // payload - final MutableLong numPayloadBytes = new MutableLong(); - subset.forAllRowKeyRanges((s, e) -> { - numPayloadBytes.add(byteStorage.getPayloadSize((int) s, (int) e)); - }); - final long payloadExtended = numPayloadBytes.get() & REMAINDER_MOD_8_MASK; - if (payloadExtended > 0) { - numPayloadBytes.add(8 - payloadExtended); - } - listener.noteLogicalBuffer(numPayloadBytes.get()); - } - - @Override - protected int getRawSize() { - if (cachedSize == -1) { - MutableLong totalCachedSize = new MutableLong(0L); - if (sendValidityBuffer()) { - totalCachedSize.add(getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME))); - } - - // there are n+1 offsets; it is not assumed first offset is zero - if (!subset.isEmpty() && subset.size() == byteStorage.offsets.size() - 1) { - totalCachedSize.add(byteStorage.offsets.size() * (long) Integer.BYTES); - totalCachedSize.add(byteStorage.size()); - } else { - totalCachedSize.add(subset.isEmpty() ? 0 : Integer.BYTES); // account for the n+1 offset - subset.forAllRowKeyRanges((s, e) -> { - // account for offsets - totalCachedSize.add((e - s + 1) * Integer.BYTES); - - // account for payload - totalCachedSize.add(byteStorage.getPayloadSize((int) s, (int) e)); - }); - } - - if (!subset.isEmpty() && (subset.size() & 0x1) == 0) { - // then we must also align offset array - totalCachedSize.add(Integer.BYTES); - } - cachedSize = LongSizedDataStructure.intSize(DEBUG_NAME, totalCachedSize.get()); - } - return cachedSize; - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - read = true; - long bytesWritten = 0; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException("couldn't drain data to OutputStream", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(rawRow -> { - final int row = LongSizedDataStructure.intSize(DEBUG_NAME, rawRow); - if (chunk.get(row) != null) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)); - } - - // write offsets array - dos.writeInt(0); - - final MutableInt logicalSize = new MutableInt(); - subset.forAllRowKeys((idx) -> { - try { - logicalSize.add(LongSizedDataStructure.intSize("int cast", - byteStorage.getPayloadSize((int) idx, (int) idx))); - dos.writeInt(logicalSize.get()); - } catch (final IOException e) { - throw new UncheckedDeephavenException("couldn't drain data to OutputStream", e); - } - }); - bytesWritten += Integer.BYTES * (subset.size() + 1); - - if ((subset.size() & 0x1) == 0) { - // then we must pad to align next buffer - dos.writeInt(0); - bytesWritten += Integer.BYTES; - } - - final MutableLong payloadLen = new MutableLong(); - subset.forAllRowKeyRanges((s, e) -> { - try { - payloadLen.add(byteStorage.writePayload(dos, (int) s, (int) e)); - } catch (final IOException err) { - throw new UncheckedDeephavenException("couldn't drain data to OutputStream", err); - } - }); - bytesWritten += payloadLen.get(); - - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); - } - } - - public static WritableObjectChunk extractChunkFromInputStream( - final DataInput is, - final Iterator fieldNodeIter, - final PrimitiveIterator.OfLong bufferInfoIter, - final Mapper mapper, - final WritableChunk outChunk, - final int outOffset, - final int totalRows) throws IOException { - final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.nextLong(); - final long offsetsBuffer = bufferInfoIter.nextLong(); - final long payloadBuffer = bufferInfoIter.nextLong(); - - final int numElements = nodeInfo.numElements; - final WritableObjectChunk chunk; - if (outChunk != null) { - chunk = outChunk.asWritableObjectChunk(); - } else { - final int numRows = Math.max(totalRows, numElements); - chunk = WritableObjectChunk.makeWritableChunk(numRows); - chunk.setSize(numRows); - } - - if (numElements == 0) { - return chunk; - } - - final int numValidityWords = (numElements + 63) / 64; - try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityWords); - final WritableIntChunk offsets = WritableIntChunk.makeWritableChunk(numElements + 1)) { - // Read validity buffer: - int jj = 0; - for (; jj < Math.min(numValidityWords, validityBuffer / 8); ++jj) { - isValid.set(jj, is.readLong()); - } - final long valBufRead = jj * 8L; - if (valBufRead < validityBuffer) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, validityBuffer - valBufRead)); - } - // we support short validity buffers - for (; jj < numValidityWords; ++jj) { - isValid.set(jj, -1); // -1 is bit-wise representation of all ones - } - - // Read offsets: - final long offBufRead = (numElements + 1L) * Integer.BYTES; - if (offsetsBuffer < offBufRead) { - throw new IllegalStateException("offset buffer is too short for the expected number of elements"); - } - for (int i = 0; i < numElements + 1; ++i) { - offsets.set(i, is.readInt()); - } - if (offBufRead < offsetsBuffer) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, offsetsBuffer - offBufRead)); - } - - // Read data: - final int bytesRead = LongSizedDataStructure.intSize(DEBUG_NAME, payloadBuffer); - final byte[] serializedData = new byte[bytesRead]; - is.readFully(serializedData); - - // Deserialize: - int ei = 0; - int pendingSkips = 0; - - for (int vi = 0; vi < numValidityWords; ++vi) { - int bitsLeftInThisWord = Math.min(64, numElements - vi * 64); - long validityWord = isValid.get(vi); - do { - if ((validityWord & 1) == 1) { - if (pendingSkips > 0) { - chunk.fillWithNullValue(outOffset + ei, pendingSkips); - ei += pendingSkips; - pendingSkips = 0; - } - final int offset = offsets.get(ei); - final int length = offsets.get(ei + 1) - offset; - Assert.geq(length, "length", 0); - if (offset + length > serializedData.length) { - throw new IllegalStateException("not enough data was serialized to parse this element: " + - "elementIndex=" + ei + " offset=" + offset + " length=" + length + - " serializedLen=" + serializedData.length); - } - chunk.set(outOffset + ei++, mapper.constructFrom(serializedData, offset, length)); - validityWord >>= 1; - bitsLeftInThisWord--; - } else { - final int skips = Math.min(Long.numberOfTrailingZeros(validityWord), bitsLeftInThisWord); - pendingSkips += skips; - validityWord >>= skips; - bitsLeftInThisWord -= skips; - } - } while (bitsLeftInThisWord > 0); - } - - if (pendingSkips > 0) { - chunk.fillWithNullValue(outOffset + ei, pendingSkips); - } - } - - return chunk; - } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java deleted file mode 100644 index d85c2d6c3d4..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java +++ /dev/null @@ -1,235 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.extensions.barrage.chunk.array.ArrayExpansionKernel; -import io.deephaven.util.mutable.MutableInt; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -public class VarListChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { - private static final String DEBUG_NAME = "VarListChunkInputStreamGenerator"; - - private final Factory factory; - private final Class type; - - private WritableIntChunk offsets; - private ChunkInputStreamGenerator innerGenerator; - - VarListChunkInputStreamGenerator(ChunkInputStreamGenerator.Factory factory, final Class type, - final ObjectChunk chunk, final long rowOffset) { - super(chunk, 0, rowOffset); - this.factory = factory; - this.type = type; - } - - private synchronized void computePayload() { - if (innerGenerator != null) { - return; - } - - final Class myType = type.getComponentType(); - final Class myComponentType = myType != null ? myType.getComponentType() : null; - - final ChunkType chunkType; - if (myType == boolean.class || myType == Boolean.class) { - // Note: Internally booleans are passed around as bytes, but the wire format is packed bits. - chunkType = ChunkType.Byte; - } else if (myType != null && !myType.isPrimitive()) { - chunkType = ChunkType.Object; - } else { - chunkType = ChunkType.fromElementType(myType); - } - - final ArrayExpansionKernel kernel = ArrayExpansionKernel.makeExpansionKernel(chunkType, myType); - offsets = WritableIntChunk.makeWritableChunk(chunk.size() + 1); - - final WritableChunk innerChunk = kernel.expand(chunk, offsets); - innerGenerator = factory.makeInputStreamGenerator(chunkType, myType, myComponentType, innerChunk, 0); - } - - @Override - protected void onReferenceCountAtZero() { - super.onReferenceCountAtZero(); - if (offsets != null) { - offsets.close(); - } - if (innerGenerator != null) { - innerGenerator.close(); - } - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, - @Nullable final RowSet subset) throws IOException { - computePayload(); - return new VarListInputStream(options, subset); - } - - private class VarListInputStream extends BaseChunkInputStream { - private int cachedSize = -1; - private final WritableIntChunk myOffsets; - private final DrainableColumn innerStream; - - private VarListInputStream( - final StreamReaderOptions options, final RowSet subsetIn) throws IOException { - super(chunk, options, subsetIn); - if (subset.size() != offsets.size() - 1) { - myOffsets = WritableIntChunk.makeWritableChunk(subset.intSize(DEBUG_NAME) + 1); - myOffsets.set(0, 0); - final RowSetBuilderSequential myOffsetBuilder = RowSetFactory.builderSequential(); - final MutableInt off = new MutableInt(); - subset.forAllRowKeys(key -> { - final int startOffset = offsets.get(LongSizedDataStructure.intSize(DEBUG_NAME, key)); - final int endOffset = offsets.get(LongSizedDataStructure.intSize(DEBUG_NAME, key + 1)); - final int idx = off.incrementAndGet(); - myOffsets.set(idx, endOffset - startOffset + myOffsets.get(idx - 1)); - if (endOffset > startOffset) { - myOffsetBuilder.appendRange(startOffset, endOffset - 1); - } - }); - try (final RowSet mySubset = myOffsetBuilder.build()) { - innerStream = innerGenerator.getInputStream(options, mySubset); - } - } else { - myOffsets = null; - innerStream = innerGenerator.getInputStream(options, null); - } - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(i -> { - if (chunk.get((int) i) == null) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - innerStream.visitFieldNodes(listener); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - final int numElements = subset.intSize(DEBUG_NAME); - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(numElements) : 0); - - // offsets - long numOffsetBytes = Integer.BYTES * (((long) numElements) + (numElements > 0 ? 1 : 0)); - final long bytesExtended = numOffsetBytes & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - numOffsetBytes += 8 - bytesExtended; - } - listener.noteLogicalBuffer(numOffsetBytes); - - // payload - innerStream.visitBuffers(listener); - } - - @Override - public void close() throws IOException { - super.close(); - if (myOffsets != null) { - myOffsets.close(); - } - innerStream.close(); - } - - @Override - protected int getRawSize() throws IOException { - if (cachedSize == -1) { - // there are n+1 offsets; it is not assumed first offset is zero - cachedSize = sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)) : 0; - cachedSize += subset.size() * Integer.BYTES + (subset.isEmpty() ? 0 : Integer.BYTES); - - if (!subset.isEmpty() && (subset.size() & 0x1) == 0) { - // then we must also align offset array - cachedSize += Integer.BYTES; - } - cachedSize += innerStream.available(); - } - return cachedSize; - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - read = true; - long bytesWritten = 0; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException("couldn't drain data to OutputStream", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(rawRow -> { - final int row = LongSizedDataStructure.intSize(DEBUG_NAME, rawRow); - if (chunk.get(row) != null) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)); - } - - // write offsets array - final WritableIntChunk offsetsToUse = myOffsets == null ? offsets : myOffsets; - for (int i = 0; i < offsetsToUse.size(); ++i) { - dos.writeInt(offsetsToUse.get(i)); - } - bytesWritten += ((long) offsetsToUse.size()) * Integer.BYTES; - - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - bytesWritten += innerStream.drainTo(outputStream); - return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); - } - } - -} - diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkReader.java deleted file mode 100644 index 4e5b8cb0bd7..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkReader.java +++ /dev/null @@ -1,116 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.chunk.array.ArrayExpansionKernel; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; - -import java.io.DataInput; -import java.io.IOException; -import java.util.Iterator; -import java.util.PrimitiveIterator; - -import static io.deephaven.extensions.barrage.chunk.ChunkReader.typeInfo; - -public class VarListChunkReader implements ChunkReader { - private static final String DEBUG_NAME = "VarListChunkReader"; - - private final ArrayExpansionKernel kernel; - private final ChunkReader componentReader; - - public VarListChunkReader(final StreamReaderOptions options, final TypeInfo typeInfo, - Factory chunkReaderFactory) { - final Class componentType = typeInfo.type().getComponentType(); - final Class innerComponentType = componentType != null ? componentType.getComponentType() : null; - - final ChunkType chunkType; - if (componentType == boolean.class || componentType == Boolean.class) { - // Note: Internally booleans are passed around as bytes, but the wire format is packed bits. - chunkType = ChunkType.Byte; - } else if (componentType != null && !componentType.isPrimitive()) { - chunkType = ChunkType.Object; - } else { - chunkType = ChunkType.fromElementType(componentType); - } - kernel = ArrayExpansionKernel.makeExpansionKernel(chunkType, componentType); - - componentReader = chunkReaderFactory.getReader(options, - typeInfo(chunkType, componentType, innerComponentType, typeInfo.componentArrowField())); - } - - @Override - public WritableObjectChunk readChunk(Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.nextLong(); - final long offsetsBuffer = bufferInfoIter.nextLong(); - - if (nodeInfo.numElements == 0) { - try (final WritableChunk ignored = - componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { - return WritableObjectChunk.makeWritableChunk(nodeInfo.numElements); - } - } - - final WritableObjectChunk chunk; - final int numValidityLongs = (nodeInfo.numElements + 63) / 64; - try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs); - final WritableIntChunk offsets = - WritableIntChunk.makeWritableChunk(nodeInfo.numElements + 1)) { - // Read validity buffer: - int jj = 0; - for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { - isValid.set(jj, is.readLong()); - } - final long valBufRead = jj * 8L; - if (valBufRead < validityBuffer) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, validityBuffer - valBufRead)); - } - // we support short validity buffers - for (; jj < numValidityLongs; ++jj) { - isValid.set(jj, -1); // -1 is bit-wise representation of all ones - } - // consumed entire validity buffer by here - - // Read offsets: - final long offBufRead = (nodeInfo.numElements + 1L) * Integer.BYTES; - if (offsetsBuffer < offBufRead) { - throw new IllegalStateException("offset buffer is too short for the expected number of elements"); - } - for (int i = 0; i < nodeInfo.numElements + 1; ++i) { - offsets.set(i, is.readInt()); - } - if (offBufRead < offsetsBuffer) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, offsetsBuffer - offBufRead)); - } - - try (final WritableChunk inner = - componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { - chunk = kernel.contract(inner, offsets, outChunk, outOffset, totalRows); - - long nextValid = 0; - for (int ii = 0; ii < nodeInfo.numElements; ++ii) { - if ((ii % 64) == 0) { - nextValid = isValid.get(ii / 64); - } - if ((nextValid & 0x1) == 0x0) { - chunk.set(outOffset + ii, null); - } - nextValid >>= 1; - } - } - } - - return chunk; - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java deleted file mode 100644 index 6b15bb348a4..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java +++ /dev/null @@ -1,227 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.extensions.barrage.chunk.vector.VectorExpansionKernel; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.vector.Vector; -import io.deephaven.util.mutable.MutableInt; -import org.jetbrains.annotations.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -public class VectorChunkInputStreamGenerator extends BaseChunkInputStreamGenerator, Values>> { - private static final String DEBUG_NAME = "VarListChunkInputStreamGenerator"; - - private final Class componentType; - private final Factory factory; - - private WritableIntChunk offsets; - private ChunkInputStreamGenerator innerGenerator; - - VectorChunkInputStreamGenerator( - final ChunkInputStreamGenerator.Factory factory, - final Class> type, - final Class componentType, - final ObjectChunk, Values> chunk, - final long rowOffset) { - super(chunk, 0, rowOffset); - this.factory = factory; - this.componentType = VectorExpansionKernel.getComponentType(type, componentType); - } - - private synchronized void computePayload() { - if (innerGenerator != null) { - return; - } - - final Class innerComponentType = componentType != null ? componentType.getComponentType() : null; - final ChunkType chunkType = ChunkType.fromElementType(componentType); - final VectorExpansionKernel kernel = VectorExpansionKernel.makeExpansionKernel(chunkType, componentType); - offsets = WritableIntChunk.makeWritableChunk(chunk.size() + 1); - - final WritableChunk innerChunk = kernel.expand(chunk, offsets); - innerGenerator = factory.makeInputStreamGenerator(chunkType, componentType, innerComponentType, innerChunk, 0); - } - - @Override - protected void onReferenceCountAtZero() { - super.onReferenceCountAtZero(); - if (offsets != null) { - offsets.close(); - } - if (innerGenerator != null) { - innerGenerator.close(); - } - } - - @Override - public DrainableColumn getInputStream(final StreamReaderOptions options, - @Nullable final RowSet subset) throws IOException { - computePayload(); - return new VarListInputStream(options, subset); - } - - private class VarListInputStream extends BaseChunkInputStream { - private int cachedSize = -1; - private final WritableIntChunk myOffsets; - private final DrainableColumn innerStream; - - private VarListInputStream( - final StreamReaderOptions options, final RowSet subsetIn) throws IOException { - super(chunk, options, subsetIn); - if (subset.size() != offsets.size() - 1) { - myOffsets = WritableIntChunk.makeWritableChunk(subset.intSize(DEBUG_NAME) + 1); - myOffsets.set(0, 0); - final RowSetBuilderSequential myOffsetBuilder = RowSetFactory.builderSequential(); - final MutableInt off = new MutableInt(); - subset.forAllRowKeys(key -> { - final int startOffset = offsets.get(LongSizedDataStructure.intSize(DEBUG_NAME, key)); - final int endOffset = offsets.get(LongSizedDataStructure.intSize(DEBUG_NAME, key + 1)); - final int idx = off.incrementAndGet(); - myOffsets.set(idx, endOffset - startOffset + myOffsets.get(idx - 1)); - if (endOffset > startOffset) { - myOffsetBuilder.appendRange(startOffset, endOffset - 1); - } - }); - try (final RowSet mySubset = myOffsetBuilder.build()) { - innerStream = innerGenerator.getInputStream(options, mySubset); - } - } else { - myOffsets = null; - innerStream = innerGenerator.getInputStream(options, null); - } - } - - private int cachedNullCount = -1; - - @Override - public int nullCount() { - if (cachedNullCount == -1) { - cachedNullCount = 0; - subset.forAllRowKeys(i -> { - if (chunk.get((int) i) == null) { - ++cachedNullCount; - } - }); - } - return cachedNullCount; - } - - @Override - public void visitFieldNodes(final FieldNodeListener listener) { - listener.noteLogicalFieldNode(subset.intSize(DEBUG_NAME), nullCount()); - innerStream.visitFieldNodes(listener); - } - - @Override - public void visitBuffers(final BufferListener listener) { - // validity - final int numElements = subset.intSize(DEBUG_NAME); - listener.noteLogicalBuffer(sendValidityBuffer() ? getValidityMapSerializationSizeFor(numElements) : 0); - - // offsets - long numOffsetBytes = Integer.BYTES * (((long) numElements) + (numElements > 0 ? 1 : 0)); - final long bytesExtended = numOffsetBytes & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - numOffsetBytes += 8 - bytesExtended; - } - listener.noteLogicalBuffer(numOffsetBytes); - - // payload - innerStream.visitBuffers(listener); - } - - @Override - public void close() throws IOException { - super.close(); - if (myOffsets != null) { - myOffsets.close(); - } - innerStream.close(); - } - - @Override - protected int getRawSize() throws IOException { - if (cachedSize == -1) { - // there are n+1 offsets; it is not assumed first offset is zero - cachedSize = sendValidityBuffer() ? getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)) : 0; - cachedSize += subset.size() * Integer.BYTES + (subset.isEmpty() ? 0 : Integer.BYTES); - - if (!subset.isEmpty() && (subset.size() & 0x1) == 0) { - // then we must also align offset array - cachedSize += Integer.BYTES; - } - cachedSize += innerStream.available(); - } - return cachedSize; - } - - @Override - public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { - return 0; - } - - read = true; - long bytesWritten = 0; - final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); - // write the validity array with LSB indexing - if (sendValidityBuffer()) { - final SerContext context = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(context.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException("couldn't drain data to OutputStream", e); - } - context.accumulator = 0; - context.count = 0; - }; - subset.forAllRowKeys(rawRow -> { - final int row = LongSizedDataStructure.intSize(DEBUG_NAME, rawRow); - if (chunk.get(row) != null) { - context.accumulator |= 1L << context.count; - } - if (++context.count == 64) { - flush.run(); - } - }); - if (context.count > 0) { - flush.run(); - } - bytesWritten += getValidityMapSerializationSizeFor(subset.intSize(DEBUG_NAME)); - } - - // write offsets array - final WritableIntChunk offsetsToUse = myOffsets == null ? offsets : myOffsets; - for (int i = 0; i < offsetsToUse.size(); ++i) { - dos.writeInt(offsetsToUse.get(i)); - } - bytesWritten += ((long) offsetsToUse.size()) * Integer.BYTES; - - final long bytesExtended = bytesWritten & REMAINDER_MOD_8_MASK; - if (bytesExtended > 0) { - bytesWritten += 8 - bytesExtended; - dos.write(PADDING_BUFFER, 0, (int) (8 - bytesExtended)); - } - - bytesWritten += innerStream.drainTo(outputStream); - return LongSizedDataStructure.intSize(DEBUG_NAME, bytesWritten); - } - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkReader.java deleted file mode 100644 index 4832ae3baa6..00000000000 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkReader.java +++ /dev/null @@ -1,112 +0,0 @@ -// -// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending -// -package io.deephaven.extensions.barrage.chunk; - -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.chunk.vector.VectorExpansionKernel; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; -import io.deephaven.util.datastructures.LongSizedDataStructure; -import io.deephaven.vector.Vector; - -import java.io.DataInput; -import java.io.IOException; -import java.util.Iterator; -import java.util.PrimitiveIterator; - -import static io.deephaven.extensions.barrage.chunk.ChunkReader.typeInfo; - -public class VectorChunkReader implements ChunkReader { - private static final String DEBUG_NAME = "VectorChunkReader"; - private final ChunkReader componentReader; - private final VectorExpansionKernel kernel; - - public VectorChunkReader(final StreamReaderOptions options, final TypeInfo typeInfo, - Factory chunkReaderFactory) { - - final Class componentType = - VectorExpansionKernel.getComponentType(typeInfo.type(), typeInfo.componentType()); - final ChunkType chunkType = ChunkType.fromElementType(componentType); - componentReader = chunkReaderFactory.getReader( - options, typeInfo(chunkType, componentType, componentType.getComponentType(), - typeInfo.componentArrowField())); - kernel = VectorExpansionKernel.makeExpansionKernel(chunkType, componentType); - } - - @Override - public WritableObjectChunk, Values> readChunk( - Iterator fieldNodeIter, - PrimitiveIterator.OfLong bufferInfoIter, DataInput is, WritableChunk outChunk, int outOffset, - int totalRows) throws IOException { - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.nextLong(); - final long offsetsBuffer = bufferInfoIter.nextLong(); - - if (nodeInfo.numElements == 0) { - try (final WritableChunk ignored = - componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { - if (outChunk != null) { - return outChunk.asWritableObjectChunk(); - } - return WritableObjectChunk.makeWritableChunk(totalRows); - } - } - - final WritableObjectChunk, Values> chunk; - final int numValidityLongs = (nodeInfo.numElements + 63) / 64; - try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs); - final WritableIntChunk offsets = - WritableIntChunk.makeWritableChunk(nodeInfo.numElements + 1)) { - // Read validity buffer: - int jj = 0; - for (; jj < Math.min(numValidityLongs, validityBuffer / 8); ++jj) { - isValid.set(jj, is.readLong()); - } - final long valBufRead = jj * 8L; - if (valBufRead < validityBuffer) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, validityBuffer - valBufRead)); - } - // we support short validity buffers - for (; jj < numValidityLongs; ++jj) { - isValid.set(jj, -1); // -1 is bit-wise representation of all ones - } - // consumed entire validity buffer by here - - // Read offsets: - final long offBufRead = (nodeInfo.numElements + 1L) * Integer.BYTES; - if (offsetsBuffer < offBufRead) { - throw new IllegalStateException("offset buffer is too short for the expected number of elements"); - } - for (int i = 0; i < nodeInfo.numElements + 1; ++i) { - offsets.set(i, is.readInt()); - } - if (offBufRead < offsetsBuffer) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, offsetsBuffer - offBufRead)); - } - - try (final WritableChunk inner = - componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { - chunk = kernel.contract(inner, offsets, outChunk, outOffset, totalRows); - - long nextValid = 0; - for (int ii = 0; ii < nodeInfo.numElements; ++ii) { - if ((ii % 64) == 0) { - nextValid = isValid.get(ii / 64); - } - if ((nextValid & 0x1) == 0x0) { - chunk.set(outOffset + ii, null); - } - nextValid >>= 1; - } - } - } - - return chunk; - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ArrayExpansionKernel.java index daa293f562d..4ad85fec0aa 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ArrayExpansionKernel.java @@ -3,73 +3,39 @@ // package io.deephaven.extensions.barrage.chunk.array; -import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.Any; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.extensions.barrage.chunk.ExpansionKernel; -public interface ArrayExpansionKernel { +public interface ArrayExpansionKernel extends ExpansionKernel { /** * @return a kernel that expands a {@code Chunk} to pair of {@code LongChunk, Chunk} */ - static ArrayExpansionKernel makeExpansionKernel(final ChunkType chunkType, final Class componentType) { + @SuppressWarnings("unchecked") + static ArrayExpansionKernel makeExpansionKernel(final ChunkType chunkType, final Class componentType) { + // Note: Internally booleans are passed around as bytes, but the wire format is packed bits. + if (componentType == boolean.class) { + return (ArrayExpansionKernel) BooleanArrayExpansionKernel.INSTANCE; + } else if (componentType == Boolean.class) { + return (ArrayExpansionKernel) BoxedBooleanArrayExpansionKernel.INSTANCE; + } + switch (chunkType) { case Char: - return CharArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) CharArrayExpansionKernel.INSTANCE; case Byte: - // Note: Internally booleans are passed around as bytes, but the wire format is packed bits. - if (componentType == boolean.class) { - return BooleanArrayExpansionKernel.INSTANCE; - } else if (componentType == Boolean.class) { - return BoxedBooleanArrayExpansionKernel.INSTANCE; - } - return ByteArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) ByteArrayExpansionKernel.INSTANCE; case Short: - return ShortArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) ShortArrayExpansionKernel.INSTANCE; case Int: - return IntArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) IntArrayExpansionKernel.INSTANCE; case Long: - return LongArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) LongArrayExpansionKernel.INSTANCE; case Float: - return FloatArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) FloatArrayExpansionKernel.INSTANCE; case Double: - return DoubleArrayExpansionKernel.INSTANCE; + return (ArrayExpansionKernel) DoubleArrayExpansionKernel.INSTANCE; default: - return new ObjectArrayExpansionKernel(componentType); + return (ArrayExpansionKernel) new ObjectArrayExpansionKernel<>(componentType); } } - - /** - * This expands the source from a {@code T[]} per element to a flat {@code T} per element. The kernel records the - * number of consecutive elements that belong to a row in {@code perElementLengthDest}. The returned chunk is owned - * by the caller. - * - * @param source the source chunk of T[] to expand - * @param perElementLengthDest the destination IntChunk for which {@code dest.get(i + 1) - dest.get(i)} is - * equivalent to {@code source.get(i).length} - * @return an unrolled/flattened chunk of T - */ - WritableChunk expand(ObjectChunk source, - WritableIntChunk perElementLengthDest); - - /** - * This contracts the source from a pair of {@code LongChunk} and {@code Chunk} and produces a - * {@code Chunk}. The returned chunk is owned by the caller. - * - * @param source the source chunk of T to contract - * @param perElementLengthDest the source IntChunk for which {@code dest.get(i + 1) - dest.get(i)} is equivalent to - * {@code source.get(i).length} - * @param outChunk the returned chunk from an earlier record batch - * @param outOffset the offset to start writing into {@code outChunk} - * @param totalRows the total known rows for this column; if known (else 0) - * @return a result chunk of T[] - */ - WritableObjectChunk contract( - Chunk source, IntChunk perElementLengthDest, WritableChunk outChunk, int outOffset, - int totalRows); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BooleanArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BooleanArrayExpansionKernel.java index 9e6cd453690..3cee60be8a0 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BooleanArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BooleanArrayExpansionKernel.java @@ -12,37 +12,47 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.BooleanUtils; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class BooleanArrayExpansionKernel implements ArrayExpansionKernel { +public class BooleanArrayExpansionKernel implements ArrayExpansionKernel { private final static boolean[] ZERO_LEN_ARRAY = new boolean[0]; public final static BooleanArrayExpansionKernel INSTANCE = new BooleanArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableByteChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final boolean[] row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final boolean[] row = typedSource.get(ii); totalSize += row == null ? 0 : row.length; } final WritableByteChunk result = WritableByteChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final boolean[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final boolean[] row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } @@ -52,25 +62,34 @@ public WritableChunk expand(final ObjectChunk source } lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ByteChunk typedSource = source.asByteChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,21 +99,20 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final boolean[] row = new boolean[rowLen]; for (int j = 0; j < rowLen; ++j) { row[j] = typedSource.get(lenRead + j) > 0; } lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BoxedBooleanArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BoxedBooleanArrayExpansionKernel.java index 0a02ddb31f9..c37a2c8fa74 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BoxedBooleanArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/BoxedBooleanArrayExpansionKernel.java @@ -12,37 +12,47 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.BooleanUtils; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class BoxedBooleanArrayExpansionKernel implements ArrayExpansionKernel { +public class BoxedBooleanArrayExpansionKernel implements ArrayExpansionKernel { private final static Boolean[] ZERO_LEN_ARRAY = new Boolean[0]; public final static BoxedBooleanArrayExpansionKernel INSTANCE = new BoxedBooleanArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableByteChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final Boolean[] row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final Boolean[] row = typedSource.get(ii); totalSize += row == null ? 0 : row.length; } final WritableByteChunk result = WritableByteChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final Boolean[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final Boolean[] row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } @@ -52,25 +62,34 @@ public WritableChunk expand(final ObjectChunk source } lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ByteChunk typedSource = source.asByteChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,21 +99,20 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final Boolean[] row = new Boolean[rowLen]; for (int j = 0; j < rowLen; ++j) { row[j] = BooleanUtils.byteAsBoolean(typedSource.get(lenRead + j)); } lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ByteArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ByteArrayExpansionKernel.java index 92e099af9e2..df5edb14662 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ByteArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ByteArrayExpansionKernel.java @@ -16,61 +16,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class ByteArrayExpansionKernel implements ArrayExpansionKernel { +public class ByteArrayExpansionKernel implements ArrayExpansionKernel { private final static byte[] ZERO_LEN_ARRAY = new byte[0]; public final static ByteArrayExpansionKernel INSTANCE = new ByteArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableByteChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final byte[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final byte[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableByteChunk result = WritableByteChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final byte[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final byte[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ByteChunk typedSource = source.asByteChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,19 +100,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final byte[] row = new byte[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/CharArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/CharArrayExpansionKernel.java index 3d04e5d6057..3cd6749d0a7 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/CharArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/CharArrayExpansionKernel.java @@ -12,61 +12,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class CharArrayExpansionKernel implements ArrayExpansionKernel { +public class CharArrayExpansionKernel implements ArrayExpansionKernel { private final static char[] ZERO_LEN_ARRAY = new char[0]; public final static CharArrayExpansionKernel INSTANCE = new CharArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableCharChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final char[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final char[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableCharChunk result = WritableCharChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final char[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final char[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final CharChunk typedSource = source.asCharChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -76,19 +96,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final char[] row = new char[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/DoubleArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/DoubleArrayExpansionKernel.java index 5836b369633..be2cfcbfe27 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/DoubleArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/DoubleArrayExpansionKernel.java @@ -16,61 +16,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class DoubleArrayExpansionKernel implements ArrayExpansionKernel { +public class DoubleArrayExpansionKernel implements ArrayExpansionKernel { private final static double[] ZERO_LEN_ARRAY = new double[0]; public final static DoubleArrayExpansionKernel INSTANCE = new DoubleArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableDoubleChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final double[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final double[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableDoubleChunk result = WritableDoubleChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final double[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final double[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final DoubleChunk typedSource = source.asDoubleChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,19 +100,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final double[] row = new double[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/FloatArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/FloatArrayExpansionKernel.java index 1b3c40ef25a..7e227fa8861 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/FloatArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/FloatArrayExpansionKernel.java @@ -16,61 +16,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class FloatArrayExpansionKernel implements ArrayExpansionKernel { +public class FloatArrayExpansionKernel implements ArrayExpansionKernel { private final static float[] ZERO_LEN_ARRAY = new float[0]; public final static FloatArrayExpansionKernel INSTANCE = new FloatArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableFloatChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final float[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final float[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableFloatChunk result = WritableFloatChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final float[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final float[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final FloatChunk typedSource = source.asFloatChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,19 +100,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final float[] row = new float[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/IntArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/IntArrayExpansionKernel.java index 0d24b992456..53f6a3ba1a6 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/IntArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/IntArrayExpansionKernel.java @@ -16,61 +16,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class IntArrayExpansionKernel implements ArrayExpansionKernel { +public class IntArrayExpansionKernel implements ArrayExpansionKernel { private final static int[] ZERO_LEN_ARRAY = new int[0]; public final static IntArrayExpansionKernel INSTANCE = new IntArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableIntChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final int[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final int[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableIntChunk result = WritableIntChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final int[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final int[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final IntChunk typedSource = source.asIntChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,19 +100,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final int[] row = new int[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/LongArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/LongArrayExpansionKernel.java index 3aa6c4d5f97..da031b5f882 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/LongArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/LongArrayExpansionKernel.java @@ -16,61 +16,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class LongArrayExpansionKernel implements ArrayExpansionKernel { +public class LongArrayExpansionKernel implements ArrayExpansionKernel { private final static long[] ZERO_LEN_ARRAY = new long[0]; public final static LongArrayExpansionKernel INSTANCE = new LongArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableLongChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final long[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final long[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableLongChunk result = WritableLongChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final long[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final long[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final LongChunk typedSource = source.asLongChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,19 +100,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final long[] row = new long[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ObjectArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ObjectArrayExpansionKernel.java index f20e408bc6b..28210c017c5 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ObjectArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ObjectArrayExpansionKernel.java @@ -4,6 +4,7 @@ package io.deephaven.extensions.barrage.chunk.array; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.IntChunk; @@ -12,63 +13,84 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class ObjectArrayExpansionKernel implements ArrayExpansionKernel { +public class ObjectArrayExpansionKernel implements ArrayExpansionKernel { - private final Class componentType; + private final Class componentType; - public ObjectArrayExpansionKernel(final Class componentType) { + public ObjectArrayExpansionKernel(final Class componentType) { this.componentType = componentType; } @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableObjectChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final T[] row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final T[] row = typedSource.get(ii); totalSize += row == null ? 0 : row.length; } final WritableObjectChunk result = WritableObjectChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final T[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final T[] row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ObjectChunk typedSource = source.asObjectChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -78,17 +100,17 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); - final Object[] row = (Object[]) ArrayReflectUtil.newInstance(componentType, rowLen); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); + // noinspection unchecked + final T[] row = (T[]) ArrayReflectUtil.newInstance(componentType, rowLen); if (rowLen != 0) { typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; } - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ShortArrayExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ShortArrayExpansionKernel.java index 61b574837f7..29f81202b6e 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ShortArrayExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/array/ShortArrayExpansionKernel.java @@ -16,61 +16,81 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; -public class ShortArrayExpansionKernel implements ArrayExpansionKernel { +public class ShortArrayExpansionKernel implements ArrayExpansionKernel { private final static short[] ZERO_LEN_ARRAY = new short[0]; public final static ShortArrayExpansionKernel INSTANCE = new ShortArrayExpansionKernel(); @Override - public WritableChunk expand(final ObjectChunk source, - final WritableIntChunk perElementLengthDest) { + public WritableChunk expand( + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableShortChunk.makeWritableChunk(0); } - final ObjectChunk typedSource = source.asObjectChunk(); - long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final short[] row = typedSource.get(i); + for (int ii = 0; ii < source.size(); ++ii) { + final short[] row = source.get(ii); totalSize += row == null ? 0 : row.length; } final WritableShortChunk result = WritableShortChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); int lenWritten = 0; - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final short[] row = typedSource.get(i); - perElementLengthDest.set(i, lenWritten); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < source.size(); ++ii) { + final short[] row = source.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, lenWritten); + } if (row == null) { continue; } result.copyFromArray(row, 0, lenWritten, row.length); lenWritten += row.length; } - perElementLengthDest.set(typedSource.size(), lenWritten); + if (offsetsDest != null) { + offsetsDest.set(source.size(), lenWritten); + } return result; } @Override - public WritableObjectChunk contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (lengths != null && lengths.size() == 0 + || lengths == null && offsets != null && offsets.size() <= 1) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } - return WritableObjectChunk.makeWritableChunk(totalRows); + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(totalRows); + chunk.fillWithNullValue(0, totalRows); + return chunk; } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ShortChunk typedSource = source.asShortChunk(); - final WritableObjectChunk result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -80,19 +100,18 @@ public WritableObjectChunk contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LEN_ARRAY); + result.set(outOffset + ii, ZERO_LEN_ARRAY); } else { final short[] row = new short[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, row); + result.set(outOffset + ii, row); } } - // noinspection unchecked - return (WritableObjectChunk) result; + return result; } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ByteVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ByteVectorExpansionKernel.java index dee07985287..cc95e60e05d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ByteVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ByteVectorExpansionKernel.java @@ -16,42 +16,51 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.function.ByteConsumer; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfByte; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.ByteVector; import io.deephaven.vector.ByteVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.ByteVectorDirect.ZERO_LENGTH_VECTOR; -public class ByteVectorExpansionKernel implements VectorExpansionKernel { +public class ByteVectorExpansionKernel implements VectorExpansionKernel { public final static ByteVectorExpansionKernel INSTANCE = new ByteVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableByteChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final ByteVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final ByteVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableByteChunk result = WritableByteChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final ByteVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final ByteVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -60,25 +69,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ByteChunk typedSource = source.asByteChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -88,15 +106,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final byte[] row = new byte[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new ByteVectorDirect(row)); + result.set(outOffset + ii, new ByteVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/CharVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/CharVectorExpansionKernel.java index a32b1300ba6..b0d6089b2a5 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/CharVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/CharVectorExpansionKernel.java @@ -12,42 +12,51 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.function.CharConsumer; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfChar; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.CharVector; import io.deephaven.vector.CharVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.CharVectorDirect.ZERO_LENGTH_VECTOR; -public class CharVectorExpansionKernel implements VectorExpansionKernel { +public class CharVectorExpansionKernel implements VectorExpansionKernel { public final static CharVectorExpansionKernel INSTANCE = new CharVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableCharChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final CharVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final CharVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableCharChunk result = WritableCharChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final CharVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final CharVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -56,25 +65,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final CharChunk typedSource = source.asCharChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -84,15 +102,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final char[] row = new char[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new CharVectorDirect(row)); + result.set(outOffset + ii, new CharVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/DoubleVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/DoubleVectorExpansionKernel.java index b616e7d2ac9..bc0a726b560 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/DoubleVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/DoubleVectorExpansionKernel.java @@ -18,41 +18,50 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfDouble; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.DoubleVector; import io.deephaven.vector.DoubleVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.DoubleVectorDirect.ZERO_LENGTH_VECTOR; -public class DoubleVectorExpansionKernel implements VectorExpansionKernel { +public class DoubleVectorExpansionKernel implements VectorExpansionKernel { public final static DoubleVectorExpansionKernel INSTANCE = new DoubleVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableDoubleChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final DoubleVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final DoubleVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableDoubleChunk result = WritableDoubleChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final DoubleVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final DoubleVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -61,25 +70,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final DoubleChunk typedSource = source.asDoubleChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -89,15 +107,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final double[] row = new double[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new DoubleVectorDirect(row)); + result.set(outOffset + ii, new DoubleVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/FloatVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/FloatVectorExpansionKernel.java index ec0f3ad761b..9e0ba1818d7 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/FloatVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/FloatVectorExpansionKernel.java @@ -16,42 +16,51 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.function.FloatConsumer; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfFloat; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.FloatVector; import io.deephaven.vector.FloatVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.FloatVectorDirect.ZERO_LENGTH_VECTOR; -public class FloatVectorExpansionKernel implements VectorExpansionKernel { +public class FloatVectorExpansionKernel implements VectorExpansionKernel { public final static FloatVectorExpansionKernel INSTANCE = new FloatVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableFloatChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final FloatVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final FloatVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableFloatChunk result = WritableFloatChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final FloatVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final FloatVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -60,25 +69,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final FloatChunk typedSource = source.asFloatChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -88,15 +106,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final float[] row = new float[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new FloatVectorDirect(row)); + result.set(outOffset + ii, new FloatVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/IntVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/IntVectorExpansionKernel.java index 69141a5b014..3369d81f1bb 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/IntVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/IntVectorExpansionKernel.java @@ -18,41 +18,50 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfInt; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.IntVector; import io.deephaven.vector.IntVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.IntVectorDirect.ZERO_LENGTH_VECTOR; -public class IntVectorExpansionKernel implements VectorExpansionKernel { +public class IntVectorExpansionKernel implements VectorExpansionKernel { public final static IntVectorExpansionKernel INSTANCE = new IntVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableIntChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final IntVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final IntVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableIntChunk result = WritableIntChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final IntVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final IntVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -61,25 +70,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final IntChunk typedSource = source.asIntChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -89,15 +107,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final int[] row = new int[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new IntVectorDirect(row)); + result.set(outOffset + ii, new IntVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/LongVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/LongVectorExpansionKernel.java index 99461b3285f..4d208c394b0 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/LongVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/LongVectorExpansionKernel.java @@ -18,41 +18,50 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfLong; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.LongVector; import io.deephaven.vector.LongVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.LongVectorDirect.ZERO_LENGTH_VECTOR; -public class LongVectorExpansionKernel implements VectorExpansionKernel { +public class LongVectorExpansionKernel implements VectorExpansionKernel { public final static LongVectorExpansionKernel INSTANCE = new LongVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableLongChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final LongVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final LongVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableLongChunk result = WritableLongChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final LongVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final LongVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -61,25 +70,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final LongChunk typedSource = source.asLongChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -89,15 +107,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final long[] row = new long[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new LongVectorDirect(row)); + result.set(outOffset + ii, new LongVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ObjectVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ObjectVectorExpansionKernel.java index 8aa3ebf3664..ec081c7c9a3 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ObjectVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ObjectVectorExpansionKernel.java @@ -10,16 +10,18 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.ObjectVector; import io.deephaven.vector.ObjectVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.reflect.Array; -public class ObjectVectorExpansionKernel implements VectorExpansionKernel { +public class ObjectVectorExpansionKernel implements VectorExpansionKernel> { private final Class componentType; public ObjectVectorExpansionKernel(final Class componentType) { @@ -28,27 +30,34 @@ public ObjectVectorExpansionKernel(final Class componentType) { @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk, A> source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableObjectChunk.makeWritableChunk(0); } final ObjectChunk, A> typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final ObjectVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final ObjectVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableObjectChunk result = WritableObjectChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final ObjectVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final ObjectVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -57,25 +66,34 @@ public WritableChunk expand( iter.forEachRemaining(v -> result.add((T) v)); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk, A> contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ObjectChunk typedSource = source.asObjectChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk, A> result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -85,16 +103,17 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ObjectVectorDirect.ZERO_LENGTH_VECTOR); + // noinspection unchecked + result.set(outOffset + ii, (ObjectVector) ObjectVectorDirect.ZERO_LENGTH_VECTOR); } else { // noinspection unchecked final T[] row = (T[]) Array.newInstance(componentType, rowLen); typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new ObjectVectorDirect<>(row)); + result.set(outOffset + ii, new ObjectVectorDirect<>(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ShortVectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ShortVectorExpansionKernel.java index 0de64d22473..9300aec814b 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ShortVectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/ShortVectorExpansionKernel.java @@ -16,42 +16,51 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.engine.primitive.function.ShortConsumer; import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfShort; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.vector.ShortVector; import io.deephaven.vector.ShortVectorDirect; -import io.deephaven.vector.Vector; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static io.deephaven.vector.ShortVectorDirect.ZERO_LENGTH_VECTOR; -public class ShortVectorExpansionKernel implements VectorExpansionKernel { +public class ShortVectorExpansionKernel implements VectorExpansionKernel { public final static ShortVectorExpansionKernel INSTANCE = new ShortVectorExpansionKernel(); @Override public WritableChunk expand( - final ObjectChunk, A> source, final WritableIntChunk perElementLengthDest) { + @NotNull final ObjectChunk source, + @Nullable final WritableIntChunk offsetsDest) { if (source.size() == 0) { - perElementLengthDest.setSize(0); + if (offsetsDest != null) { + offsetsDest.setSize(0); + } return WritableShortChunk.makeWritableChunk(0); } final ObjectChunk typedSource = source.asObjectChunk(); long totalSize = 0; - for (int i = 0; i < typedSource.size(); ++i) { - final ShortVector row = typedSource.get(i); + for (int ii = 0; ii < typedSource.size(); ++ii) { + final ShortVector row = typedSource.get(ii); totalSize += row == null ? 0 : row.size(); } final WritableShortChunk result = WritableShortChunk.makeWritableChunk( LongSizedDataStructure.intSize("ExpansionKernel", totalSize)); result.setSize(0); - perElementLengthDest.setSize(source.size() + 1); - for (int i = 0; i < typedSource.size(); ++i) { - final ShortVector row = typedSource.get(i); - perElementLengthDest.set(i, result.size()); + if (offsetsDest != null) { + offsetsDest.setSize(source.size() + 1); + } + for (int ii = 0; ii < typedSource.size(); ++ii) { + final ShortVector row = typedSource.get(ii); + if (offsetsDest != null) { + offsetsDest.set(ii, result.size()); + } if (row == null) { continue; } @@ -60,25 +69,34 @@ public WritableChunk expand( iter.forEachRemaining(consumer); } } - perElementLengthDest.set(typedSource.size(), result.size()); + if (offsetsDest != null) { + offsetsDest.set(typedSource.size(), result.size()); + } return result; } @Override - public WritableObjectChunk, A> contract( - final Chunk source, final IntChunk perElementLengthDest, - final WritableChunk outChunk, final int outOffset, final int totalRows) { - if (perElementLengthDest.size() == 0) { + public WritableObjectChunk contract( + @NotNull final Chunk source, + final int sizePerElement, + @Nullable final IntChunk offsets, + @Nullable final IntChunk lengths, + @Nullable final WritableChunk outChunk, + final int outOffset, + final int totalRows) { + if (source.size() == 0) { if (outChunk != null) { return outChunk.asWritableObjectChunk(); } return WritableObjectChunk.makeWritableChunk(totalRows); } - final int itemsInBatch = perElementLengthDest.size() - 1; + final int itemsInBatch = offsets == null + ? source.size() / sizePerElement + : (offsets.size() - (lengths == null ? 1 : 0)); final ShortChunk typedSource = source.asShortChunk(); - final WritableObjectChunk, A> result; + final WritableObjectChunk result; if (outChunk != null) { result = outChunk.asWritableObjectChunk(); } else { @@ -88,15 +106,15 @@ public WritableObjectChunk, A> contract( } int lenRead = 0; - for (int i = 0; i < itemsInBatch; ++i) { - final int rowLen = perElementLengthDest.get(i + 1) - perElementLengthDest.get(i); + for (int ii = 0; ii < itemsInBatch; ++ii) { + final int rowLen = computeSize(ii, sizePerElement, offsets, lengths); if (rowLen == 0) { - result.set(outOffset + i, ZERO_LENGTH_VECTOR); + result.set(outOffset + ii, ZERO_LENGTH_VECTOR); } else { final short[] row = new short[rowLen]; typedSource.copyToArray(lenRead, row, 0, rowLen); lenRead += rowLen; - result.set(outOffset + i, new ShortVectorDirect(row)); + result.set(outOffset + ii, new ShortVectorDirect(row)); } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/VectorExpansionKernel.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/VectorExpansionKernel.java index 6b6b7c82e2c..0424f9b5a98 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/VectorExpansionKernel.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/vector/VectorExpansionKernel.java @@ -3,15 +3,8 @@ // package io.deephaven.extensions.barrage.chunk.vector; -import io.deephaven.chunk.Chunk; import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.IntChunk; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableChunk; -import io.deephaven.chunk.WritableIntChunk; -import io.deephaven.chunk.WritableObjectChunk; -import io.deephaven.chunk.attributes.Any; -import io.deephaven.chunk.attributes.ChunkPositions; +import io.deephaven.extensions.barrage.chunk.ExpansionKernel; import io.deephaven.vector.ByteVector; import io.deephaven.vector.CharVector; import io.deephaven.vector.DoubleVector; @@ -22,7 +15,7 @@ import io.deephaven.vector.ShortVector; import io.deephaven.vector.Vector; -public interface VectorExpansionKernel { +public interface VectorExpansionKernel> extends ExpansionKernel { static Class getComponentType(final Class type, final Class componentType) { if (ByteVector.class.isAssignableFrom(type)) { @@ -55,53 +48,26 @@ static Class getComponentType(final Class type, final Class componentTy /** * @return a kernel that expands a {@code Chunk} to pair of {@code LongChunk, Chunk} */ - static VectorExpansionKernel makeExpansionKernel(final ChunkType chunkType, final Class componentType) { + @SuppressWarnings("unchecked") + static > VectorExpansionKernel makeExpansionKernel( + final ChunkType chunkType, final Class componentType) { switch (chunkType) { case Char: - return CharVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) CharVectorExpansionKernel.INSTANCE; case Byte: - return ByteVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) ByteVectorExpansionKernel.INSTANCE; case Short: - return ShortVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) ShortVectorExpansionKernel.INSTANCE; case Int: - return IntVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) IntVectorExpansionKernel.INSTANCE; case Long: - return LongVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) LongVectorExpansionKernel.INSTANCE; case Float: - return FloatVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) FloatVectorExpansionKernel.INSTANCE; case Double: - return DoubleVectorExpansionKernel.INSTANCE; + return (VectorExpansionKernel) DoubleVectorExpansionKernel.INSTANCE; default: - return new ObjectVectorExpansionKernel<>(componentType); + return (VectorExpansionKernel) new ObjectVectorExpansionKernel<>(componentType); } } - - /** - * This expands the source from a {@code TVector} per element to a flat {@code T} per element. The kernel records - * the number of consecutive elements that belong to a row in {@code perElementLengthDest}. The returned chunk is - * owned by the caller. - * - * @param source the source chunk of TVector to expand - * @param perElementLengthDest the destination IntChunk for which {@code dest.get(i + 1) - dest.get(i)} is - * equivalent to {@code source.get(i).length} - * @return an unrolled/flattened chunk of T - */ - WritableChunk expand(ObjectChunk, A> source, - WritableIntChunk perElementLengthDest); - - /** - * This contracts the source from a pair of {@code LongChunk} and {@code Chunk} and produces a - * {@code Chunk}. The returned chunk is owned by the caller. - * - * @param source the source chunk of T to contract - * @param perElementLengthDest the source IntChunk for which {@code dest.get(i + 1) - dest.get(i)} is equivalent to - * {@code source.get(i).length} - * @param outChunk the returned chunk from an earlier record batch - * @param outOffset the offset to start writing into {@code outChunk} - * @param totalRows the total known rows for this column; if known (else 0) - * @return a result chunk of T[] - */ - WritableObjectChunk, A> contract( - Chunk source, IntChunk perElementLengthDest, - WritableChunk outChunk, int outOffset, int totalRows); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java index fe735a20f5b..ec0c0f9f953 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageRedirectedTable.java @@ -30,7 +30,6 @@ import org.jetbrains.annotations.Nullable; import java.util.ArrayDeque; -import java.util.BitSet; import java.util.LinkedHashMap; import java.util.Map; import java.util.concurrent.ScheduledExecutorService; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java index 514f76f05d6..1d5636cb28e 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java @@ -12,6 +12,7 @@ import io.deephaven.configuration.Configuration; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.table.impl.InstrumentedTableUpdateSource; +import io.deephaven.engine.table.impl.sources.ZonedDateTimeArraySource; import io.deephaven.engine.table.impl.util.*; import io.deephaven.engine.updategraph.LogicalClock; import io.deephaven.engine.updategraph.NotificationQueue; @@ -38,6 +39,8 @@ import org.jetbrains.annotations.Nullable; import java.time.Instant; +import java.time.ZoneId; +import java.time.ZonedDateTime; import java.util.*; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -491,8 +494,13 @@ protected static LinkedHashMap> makeColumns( final LinkedHashMap> finalColumns = new LinkedHashMap<>(numColumns); for (int ii = 0; ii < numColumns; ii++) { final ColumnDefinition column = columns.get(ii); - writableSources[ii] = ArrayBackedColumnSource.getMemoryColumnSource( - 0, column.getDataType(), column.getComponentType()); + if (column.getDataType() == ZonedDateTime.class) { + // TODO NATE NOCOMMIT: we need to get the timestamp up in here + writableSources[ii] = new ZonedDateTimeArraySource(ZoneId.systemDefault()); + } else { + writableSources[ii] = ArrayBackedColumnSource.getMemoryColumnSource( + 0, column.getDataType(), column.getComponentType()); + } finalColumns.put(column.getName(), WritableRedirectedColumnSource.maybeRedirect(emptyRowRedirection, writableSources[ii], 0)); } @@ -510,8 +518,13 @@ protected static LinkedHashMap> makeColumns( final LinkedHashMap> finalColumns = new LinkedHashMap<>(numColumns); for (int ii = 0; ii < numColumns; ii++) { final ColumnDefinition column = columns.get(ii); - writableSources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(0, column.getDataType(), - column.getComponentType()); + if (column.getDataType() == ZonedDateTime.class) { + // TODO NATE NOCOMMIT: we need to get the timestamp up in here + writableSources[ii] = new ZonedDateTimeArraySource(ZoneId.systemDefault()); + } else { + writableSources[ii] = ArrayBackedColumnSource.getMemoryColumnSource( + 0, column.getDataType(), column.getComponentType()); + } finalColumns.put(column.getName(), writableSources[ii]); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java index c57c2111a17..05dec0aca2f 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java @@ -7,14 +7,15 @@ import com.google.protobuf.CodedInputStream; import com.google.rpc.Code; import io.deephaven.UncheckedDeephavenException; -import io.deephaven.chunk.ChunkType; +import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.table.impl.util.BarrageMessage; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; import io.deephaven.extensions.barrage.chunk.ChunkReader; -import io.deephaven.extensions.barrage.chunk.DefaultChunkReadingFactory; +import io.deephaven.extensions.barrage.chunk.DefaultChunkReaderFactory; import io.deephaven.extensions.barrage.table.BarrageTable; import io.deephaven.io.streams.ByteBufferInputStream; import io.deephaven.proto.util.Exceptions; @@ -47,7 +48,7 @@ public class ArrowToTableConverter { private Class[] columnTypes; private Class[] componentTypes; protected BarrageSubscriptionOptions options = DEFAULT_SER_OPTIONS; - private final List readers = new ArrayList<>(); + private final List>> readers = new ArrayList<>(); private volatile boolean completed = false; @@ -64,7 +65,6 @@ private static BarrageProtoUtil.MessageInfo parseArrowIpcMessage(final ByteBuffe final ByteBuffer bodyBB = bb.slice(); final ByteBufferInputStream bbis = new ByteBufferInputStream(bodyBB); final CodedInputStream decoder = CodedInputStream.newInstance(bbis); - // noinspection UnstableApiUsage mi.inputStream = new LittleEndianDataInputStream( new BarrageProtoUtil.ObjectInputStreamAdapter(decoder, bodyBB.remaining())); } @@ -154,14 +154,11 @@ protected void parseSchema(final Message message) { resultTable = BarrageTable.make(null, result.tableDef, result.attributes, null); resultTable.setFlat(); - ChunkType[] columnChunkTypes = result.computeWireChunkTypes(); columnTypes = result.computeWireTypes(); componentTypes = result.computeWireComponentTypes(); for (int i = 0; i < schema.fieldsLength(); i++) { - final int factor = (result.conversionFactors == null) ? 1 : result.conversionFactors[i]; - ChunkReader reader = DefaultChunkReadingFactory.INSTANCE.getReader(options, factor, - typeInfo(columnChunkTypes[i], columnTypes[i], componentTypes[i], schema.fields(i))); - readers.add(reader); + readers.add(DefaultChunkReaderFactory.INSTANCE.newReader( + typeInfo(columnTypes[i], componentTypes[i], schema.fields(i)), options)); } // retain reference until the resultTable can be sealed @@ -175,9 +172,9 @@ protected BarrageMessage createBarrageMessage(BarrageProtoUtil.MessageInfo mi, i final BarrageMessage msg = new BarrageMessage(); final RecordBatch batch = (RecordBatch) mi.header.header(new RecordBatch()); - final Iterator fieldNodeIter = + final Iterator fieldNodeIter = new FlatBufferIteratorAdapter<>(batch.nodesLength(), - i -> new ChunkInputStreamGenerator.FieldNodeInfo(batch.nodes(i))); + i -> new ChunkWriter.FieldNodeInfo(batch.nodes(i))); final long[] bufferInfo = new long[batch.buffersLength()]; for (int bi = 0; bi < batch.buffersLength(); ++bi) { @@ -205,7 +202,8 @@ protected BarrageMessage createBarrageMessage(BarrageProtoUtil.MessageInfo mi, i msg.addColumnData[ci] = acd; msg.addColumnData[ci].data = new ArrayList<>(); try { - acd.data.add(readers.get(ci).readChunk(fieldNodeIter, bufferInfoIter, mi.inputStream, null, 0, 0)); + acd.data.add(readers.get(ci).readChunk(fieldNodeIter, bufferInfoIter, mi.inputStream, null, 0, + LongSizedDataStructure.intSize("ArrowToTableConverter", batch.length()))); } catch (final IOException unexpected) { throw new UncheckedDeephavenException(unexpected); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageMessageReader.java similarity index 67% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReader.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageMessageReader.java index be389e894b6..e5603e9ba76 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageMessageReader.java @@ -5,17 +5,17 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.engine.table.impl.util.BarrageMessage; +import io.deephaven.extensions.barrage.chunk.ChunkReader; import java.io.InputStream; -import java.util.BitSet; /** - * Thread safe re-usable reader that converts an InputStreams to BarrageMessages. - * + * A gRPC streaming reader that keeps stream specific context and converts {@link InputStream}s to + * {@link BarrageMessage}s. */ -public interface StreamReader { +public interface BarrageMessageReader { /** - * Converts an InputStream to a BarrageMessage in the context of the provided parameters. + * Converts an {@link InputStream} to a {@link BarrageMessage} in the context of the provided parameters. * * @param options the options related to parsing this message * @param columnChunkTypes the types to use for each column chunk @@ -24,10 +24,9 @@ public interface StreamReader { * @param stream the input stream that holds the message to be parsed * @return a BarrageMessage filled out by the stream's payload */ - BarrageMessage safelyParseFrom(final StreamReaderOptions options, + BarrageMessage safelyParseFrom(final ChunkReader.Options options, ChunkType[] columnChunkTypes, Class[] columnTypes, Class[] componentTypes, InputStream stream); - } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageMessageReaderImpl.java similarity index 94% rename from extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java rename to extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageMessageReaderImpl.java index 496de4ed31d..713cd8d2607 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageMessageReaderImpl.java @@ -17,10 +17,11 @@ import io.deephaven.engine.rowset.impl.ExternalizableRowSetUtils; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.RowSetShiftData; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.util.*; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; import io.deephaven.extensions.barrage.chunk.ChunkReader; -import io.deephaven.extensions.barrage.chunk.DefaultChunkReadingFactory; +import io.deephaven.extensions.barrage.chunk.DefaultChunkReaderFactory; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.ChunkType; import io.deephaven.internal.log.LoggerFactory; @@ -34,6 +35,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.BitSet; @@ -44,9 +46,9 @@ import static io.deephaven.extensions.barrage.chunk.ChunkReader.typeInfo; -public class BarrageStreamReader implements StreamReader { +public class BarrageMessageReaderImpl implements BarrageMessageReader { - private static final Logger log = LoggerFactory.getLogger(BarrageStreamReader.class); + private static final Logger log = LoggerFactory.getLogger(BarrageMessageReaderImpl.class); // We would like to use jdk.internal.util.ArraysSupport.MAX_ARRAY_LENGTH, but it is not exported private static final int MAX_CHUNK_SIZE = ArrayUtil.MAX_ARRAY_SIZE; @@ -60,15 +62,15 @@ public class BarrageStreamReader implements StreamReader { private BarrageMessage msg = null; - private final ChunkReader.Factory chunkReaderFactory = DefaultChunkReadingFactory.INSTANCE; - private final List readers = new ArrayList<>(); + private final ChunkReader.Factory chunkReaderFactory = DefaultChunkReaderFactory.INSTANCE; + private final List> readers = new ArrayList<>(); - public BarrageStreamReader(final LongConsumer deserializeTmConsumer) { + public BarrageMessageReaderImpl(final LongConsumer deserializeTmConsumer) { this.deserializeTmConsumer = deserializeTmConsumer; } @Override - public BarrageMessage safelyParseFrom(final StreamReaderOptions options, + public BarrageMessage safelyParseFrom(final ChunkReader.Options options, final ChunkType[] columnChunkTypes, final Class[] columnTypes, final Class[] componentTypes, @@ -200,12 +202,11 @@ public BarrageMessage safelyParseFrom(final StreamReaderOptions options, final RecordBatch batch = (RecordBatch) header.header(new RecordBatch()); msg.length = batch.length(); - // noinspection UnstableApiUsage try (final LittleEndianDataInputStream ois = new LittleEndianDataInputStream(new BarrageProtoUtil.ObjectInputStreamAdapter(decoder, size))) { - final Iterator fieldNodeIter = + final Iterator fieldNodeIter = new FlatBufferIteratorAdapter<>(batch.nodesLength(), - i -> new ChunkInputStreamGenerator.FieldNodeInfo(batch.nodes(i))); + i -> new ChunkWriter.FieldNodeInfo(batch.nodes(i))); final long[] bufferInfo = new long[batch.buffersLength()]; for (int bi = 0; bi < batch.buffersLength(); ++bi) { @@ -298,9 +299,9 @@ public BarrageMessage safelyParseFrom(final StreamReaderOptions options, header.header(schema); for (int i = 0; i < schema.fieldsLength(); i++) { Field field = schema.fields(i); - ChunkReader chunkReader = chunkReaderFactory.getReader(options, - typeInfo(columnChunkTypes[i], columnTypes[i], componentTypes[i], field)); - readers.add(chunkReader); + + final Class columnType = ReinterpretUtils.maybeConvertToPrimitiveDataType(columnTypes[i]); + readers.add(chunkReaderFactory.newReader(typeInfo(columnType, componentTypes[i], field), options)); } return null; } @@ -328,7 +329,6 @@ private static RowSet extractIndex(final ByteBuffer bb) throws IOException { if (bb == null) { return RowSetFactory.empty(); } - // noinspection UnstableApiUsage try (final LittleEndianDataInputStream is = new LittleEndianDataInputStream(new ByteBufferBackedInputStream(bb))) { return ExternalizableRowSetUtils.readExternalCompressedDelta(is); @@ -343,7 +343,6 @@ private static RowSetShiftData extractIndexShiftData(final ByteBuffer bb) throws final RowSetShiftData.Builder builder = new RowSetShiftData.Builder(); final RowSet sRowSet, eRowSet, dRowSet; - // noinspection UnstableApiUsage try (final LittleEndianDataInputStream is = new LittleEndianDataInputStream(new ByteBufferBackedInputStream(bb))) { sRowSet = ExternalizableRowSetUtils.readExternalCompressedDelta(is); diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageProtoUtil.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageProtoUtil.java index 3321951b76d..67c6b5b23bf 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageProtoUtil.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageProtoUtil.java @@ -43,7 +43,6 @@ public class BarrageProtoUtil { private static final Logger log = LoggerFactory.getLogger(BarrageProtoUtil.class); public static ByteBuffer toByteBuffer(final RowSet rowSet) { - // noinspection UnstableApiUsage try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); final LittleEndianDataOutputStream oos = new LittleEndianDataOutputStream(baos)) { ExternalizableRowSetUtils.writeExternalCompressedDeltas(oos, rowSet); @@ -55,7 +54,6 @@ public static ByteBuffer toByteBuffer(final RowSet rowSet) { } public static RowSet toRowSet(final ByteBuffer string) { - // noinspection UnstableApiUsage try (final InputStream bais = new ByteBufferInputStream(string); final LittleEndianDataInputStream ois = new LittleEndianDataInputStream(bais)) { return ExternalizableRowSetUtils.readExternalCompressedDelta(ois); @@ -137,7 +135,6 @@ public static final class MessageInfo { /** the parsed protobuf from the flight descriptor embedded in app_metadata */ public Flight.FlightDescriptor descriptor = null; /** the payload beyond the header metadata */ - @SuppressWarnings("UnstableApiUsage") public LittleEndianDataInputStream inputStream = null; } @@ -173,7 +170,6 @@ public static MessageInfo parseProtoMessage(final InputStream stream) throws IOE // at this point, we're in the body, we will read it and then break, the rest of the payload should // be the body size = decoder.readRawVarint32(); - // noinspection UnstableApiUsage mi.inputStream = new LittleEndianDataInputStream( new BarrageProtoUtil.ObjectInputStreamAdapter(decoder, size)); // we do not actually remove the content from our stream; prevent reading the next tag via a labeled @@ -187,7 +183,6 @@ public static MessageInfo parseProtoMessage(final InputStream stream) throws IOE } if (mi.header != null && mi.header.headerType() == MessageHeader.RecordBatch && mi.inputStream == null) { - // noinspection UnstableApiUsage mi.inputStream = new LittleEndianDataInputStream(new ByteArrayInputStream(ArrayTypeUtils.EMPTY_BYTE_ARRAY)); } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageUtil.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageUtil.java index b11cc5f2a08..1f64d7f7c4b 100755 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageUtil.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageUtil.java @@ -13,6 +13,8 @@ import io.deephaven.base.ArrayUtil; import io.deephaven.base.ClassUtil; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.configuration.Configuration; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; @@ -27,9 +29,12 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.util.BarrageMessage; import io.deephaven.engine.updategraph.impl.PeriodicUpdateGraph; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.BarragePerformanceLog; import io.deephaven.extensions.barrage.BarrageSnapshotOptions; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; +import io.deephaven.extensions.barrage.chunk.ChunkReader; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; +import io.deephaven.extensions.barrage.chunk.DefaultChunkWriterFactory; import io.deephaven.extensions.barrage.chunk.vector.VectorExpansionKernel; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; @@ -216,7 +221,7 @@ public static ByteString schemaBytes(@NotNull final ToIntFunction attributes, final boolean isFlat) { @@ -381,31 +386,8 @@ public static void putMetadata(final Map metadata, final String metadata.put(ATTR_DH_PREFIX + key, value); } - private static boolean maybeConvertForTimeUnit( - final TimeUnit unit, - final ConvertedArrowSchema result, - final int columnOffset) { - switch (unit) { - case NANOSECOND: - return true; - case MICROSECOND: - setConversionFactor(result, columnOffset, 1000); - return true; - case MILLISECOND: - setConversionFactor(result, columnOffset, 1000 * 1000); - return true; - case SECOND: - setConversionFactor(result, columnOffset, 1000 * 1000 * 1000); - return true; - default: - return false; - } - } - private static Class getDefaultType( final ArrowType arrowType, - final ConvertedArrowSchema result, - final int columnOffset, final Class explicitType) { final String exMsg = "Schema did not include `" + ATTR_DH_PREFIX + ATTR_TYPE_TAG + "` metadata for field "; switch (arrowType.getTypeID()) { @@ -432,6 +414,8 @@ private static Class getDefaultType( return int.class; case 32: return long.class; + case 64: + return BigInteger.class; } } throw Exceptions.statusRuntimeException(Code.INVALID_ARGUMENT, exMsg + @@ -439,19 +423,12 @@ private static Class getDefaultType( case Bool: return Boolean.class; case Duration: - final ArrowType.Duration durationType = (ArrowType.Duration) arrowType; - final TimeUnit durationUnit = durationType.getUnit(); - if (maybeConvertForTimeUnit(durationUnit, result, columnOffset)) { - return long.class; - } - throw Exceptions.statusRuntimeException(Code.INVALID_ARGUMENT, exMsg + - " of durationType(unit=" + durationUnit.toString() + ")"); + return long.class; case Timestamp: final ArrowType.Timestamp timestampType = (ArrowType.Timestamp) arrowType; final String tz = timestampType.getTimezone(); final TimeUnit timestampUnit = timestampType.getUnit(); - boolean conversionSuccess = maybeConvertForTimeUnit(timestampUnit, result, columnOffset); - if ((tz == null || "UTC".equals(tz)) && conversionSuccess) { + if ((tz == null || "UTC".equals(tz))) { return Instant.class; } if (explicitType != null) { @@ -486,9 +463,6 @@ private static Class getDefaultType( public static class ConvertedArrowSchema { public final int nCols; public TableDefinition tableDef; - // a multiplicative factor to apply when reading; useful for eg converting arrow timestamp time units - // to the expected nanos value for Instant. - public int[] conversionFactors; public Map attributes; public ConvertedArrowSchema(final int nCols) { @@ -503,7 +477,10 @@ public ChunkType[] computeWireChunkTypes() { } public Class[] computeWireTypes() { - return tableDef.getColumnStream().map(ColumnDefinition::getDataType).toArray(Class[]::new); + return tableDef.getColumnStream() + .map(ColumnDefinition::getDataType) + .map(ReinterpretUtils::maybeConvertToPrimitiveDataType) + .toArray(Class[]::new); } public Class[] computeWireComponentTypes() { @@ -512,17 +489,6 @@ public Class[] computeWireComponentTypes() { } } - private static void setConversionFactor( - final ConvertedArrowSchema result, - final int columnOffset, - final int factor) { - if (result.conversionFactors == null) { - result.conversionFactors = new int[result.nCols]; - Arrays.fill(result.conversionFactors, 1); - } - result.conversionFactors[columnOffset] = factor; - } - public static TableDefinition convertTableDefinition(final ExportedTableCreationResponse response) { return convertArrowSchema(SchemaHelper.flatbufSchema(response)).tableDef; } @@ -598,8 +564,8 @@ private static ConvertedArrowSchema convertArrowSchema( } }); - // this has side effects such as setting the conversion factor; must call even if dest type is well known - Class defaultType = getDefaultType(getArrowType.apply(i), result, i, type.getValue()); + // this has side effects such as type validation; must call even if dest type is well known + Class defaultType = getDefaultType(getArrowType.apply(i), type.getValue()); if (type.getValue() == null) { type.setValue(defaultType); @@ -691,13 +657,16 @@ private static boolean isTypeNativelySupported(final Class typ) { return false; } - private static Field arrowFieldFor( - final String name, final Class type, final Class componentType, final Map metadata) { + public static Field arrowFieldFor( + final String name, + final Class type, + final Class componentType, + final Map metadata) { List children = Collections.emptyList(); final FieldType fieldType = arrowFieldTypeFor(type, metadata); if (fieldType.getType().isComplex()) { - if (type.isArray()) { + if (type.isArray() || Vector.class.isAssignableFrom(type)) { children = Collections.singletonList(arrowFieldFor( "", componentType, componentType.getComponentType(), Collections.emptyMap())); } else { @@ -708,6 +677,27 @@ private static Field arrowFieldFor( return new Field(name, fieldType, children); } + public static org.apache.arrow.flatbuf.Field flatbufFieldFor( + final ColumnDefinition columnDefinition, + final Map metadata) { + return flatbufFieldFor( + columnDefinition.getName(), + columnDefinition.getDataType(), + columnDefinition.getComponentType(), + metadata); + } + + public static org.apache.arrow.flatbuf.Field flatbufFieldFor( + final String name, + final Class type, + final Class componentType, + final Map metadata) { + final Field field = arrowFieldFor(name, type, componentType, metadata); + final FlatBufferBuilder builder = new FlatBufferBuilder(); + builder.finish(field.getField(builder)); + return org.apache.arrow.flatbuf.Field.getRootAsField(builder.dataBuffer()); + } + private static FieldType arrowFieldTypeFor(final Class type, final Map metadata) { return new FieldType(true, arrowTypeFor(type), null, metadata); } @@ -736,6 +726,12 @@ private static ArrowType arrowTypeFor(Class type) { return Types.MinorType.FLOAT8.getType(); case Object: if (type.isArray()) { + if (type.getComponentType() == byte.class) { + return Types.MinorType.VARBINARY.getType(); + } + return Types.MinorType.LIST.getType(); + } + if (Vector.class.isAssignableFrom(type)) { return Types.MinorType.LIST.getType(); } if (type == LocalDate.class) { @@ -772,18 +768,26 @@ private static Field arrowFieldForVectorType( } public static void createAndSendStaticSnapshot( - BarrageStreamGenerator.Factory streamGeneratorFactory, + BarrageMessageWriter.Factory messageWriterFactory, BaseTable table, BitSet columns, RowSet viewport, boolean reverseViewport, BarrageSnapshotOptions snapshotRequestOptions, - StreamObserver listener, + StreamObserver listener, BarragePerformanceLog.SnapshotMetricsHelper metrics) { // start with small value and grow long snapshotTargetCellCount = MIN_SNAPSHOT_CELL_COUNT; double snapshotNanosPerCell = 0.0; + // noinspection unchecked + final ChunkWriter>[] chunkWriters = table.getDefinition().getColumns().stream() + .map(cd -> DefaultChunkWriterFactory.INSTANCE.newWriter(ChunkReader.typeInfo( + ReinterpretUtils.maybeConvertToPrimitiveDataType(cd.getDataType()), + cd.getComponentType(), + flatbufFieldFor(cd, Map.of())))) + .toArray(ChunkWriter[]::new); + final long columnCount = Math.max(1, columns != null ? columns.cardinality() : table.getDefinition().getColumns().size()); @@ -825,7 +829,8 @@ public static void createAndSendStaticSnapshot( // send out the data. Note that although a `BarrageUpdateMetaData` object will // be provided with each unique snapshot, vanilla Flight clients will ignore // these and see only an incoming stream of batches - try (final BarrageStreamGenerator bsg = streamGeneratorFactory.newGenerator(msg, metrics)) { + try (final BarrageMessageWriter bsg = + messageWriterFactory.newMessageWriter(msg, chunkWriters, metrics)) { if (rsIt.hasMore()) { listener.onNext(bsg.getSnapshotView(snapshotRequestOptions, snapshotViewport, false, @@ -866,21 +871,29 @@ public static void createAndSendStaticSnapshot( } public static void createAndSendSnapshot( - BarrageStreamGenerator.Factory streamGeneratorFactory, + BarrageMessageWriter.Factory streamWriterFactory, BaseTable table, BitSet columns, RowSet viewport, boolean reverseViewport, BarrageSnapshotOptions snapshotRequestOptions, - StreamObserver listener, + StreamObserver listener, BarragePerformanceLog.SnapshotMetricsHelper metrics) { // if the table is static and a full snapshot is requested, we can make and send multiple // snapshots to save memory and operate more efficiently if (!table.isRefreshing()) { - createAndSendStaticSnapshot(streamGeneratorFactory, table, columns, viewport, reverseViewport, + createAndSendStaticSnapshot(streamWriterFactory, table, columns, viewport, reverseViewport, snapshotRequestOptions, listener, metrics); return; } + // noinspection unchecked + final ChunkWriter>[] chunkWriters = table.getDefinition().getColumns().stream() + .map(cd -> DefaultChunkWriterFactory.INSTANCE.newWriter(ChunkReader.typeInfo( + ReinterpretUtils.maybeConvertToPrimitiveDataType(cd.getDataType()), + cd.getComponentType(), + flatbufFieldFor(cd, Map.of())))) + .toArray(ChunkWriter[]::new); + // otherwise snapshot the entire request and send to the client final BarrageMessage msg; @@ -897,7 +910,7 @@ public static void createAndSendSnapshot( msg.modColumnData = BarrageMessage.ZERO_MOD_COLUMNS; // no mod column data // translate the viewport to keyspace and make the call - try (final BarrageStreamGenerator bsg = streamGeneratorFactory.newGenerator(msg, metrics); + try (final BarrageMessageWriter bsg = streamWriterFactory.newMessageWriter(msg, chunkWriters, metrics); final RowSet keySpaceViewport = viewport != null ? msg.rowsAdded.subSetForPositions(viewport, reverseViewport) : null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/Float16.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/Float16.java new file mode 100644 index 00000000000..06d4edb9748 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/Float16.java @@ -0,0 +1,168 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.util; + +/** + * Lifted from Apache Arrow project: + * https://github.com/apache/arrow/blob/ee62d970338f173fff4c0d11b975fe30b5fda70b/java/memory/memory-core/src/main/java/org/apache/arrow/memory/util/Float16.java + * + *

+ * + * The class is a utility class to manipulate half-precision 16-bit + * IEEE 754 floating point data types + * (also called fp16 or binary16). A half-precision float can be created from or converted to single-precision floats, + * and is stored in a short data type. The IEEE 754 standard specifies an float16 as having the following format: + * + *
    + *
  • Sign bit: 1 bit + *
  • Exponent width: 5 bits + *
  • Significand: 10 bits + *
+ * + *

+ * The format is laid out as follows: + * + *

+ * 1   11111   1111111111
+ * ^   --^--   -----^----
+ * sign  |          |_______ significand
+ *       |
+ *      -- exponent
+ * 
+ * + * Half-precision floating points can be useful to save memory and/or bandwidth at the expense of range and precision + * when compared to single-precision floating points (float32). Ref: + * https://android.googlesource.com/platform/libcore/+/master/luni/src/main/java/libcore/util/FP16.java + */ +public class Float16 { + // The bitmask to and a number with to obtain the sign bit. + private static final int SIGN_MASK = 0x8000; + // The offset to shift by to obtain the exponent bits. + private static final int EXPONENT_SHIFT = 10; + // The bitmask to and a number shifted by EXPONENT_SHIFT right, to obtain exponent bits. + private static final int SHIFTED_EXPONENT_MASK = 0x1f; + // The bitmask to and a number with to obtain significand bits. + private static final int SIGNIFICAND_MASK = 0x3ff; + // The offset of the exponent from the actual value. + private static final int EXPONENT_BIAS = 15; + // The offset to shift by to obtain the sign bit. + private static final int SIGN_SHIFT = 15; + + private static final int FP32_SIGN_SHIFT = 31; + private static final int FP32_EXPONENT_SHIFT = 23; + private static final int FP32_SHIFTED_EXPONENT_MASK = 0xff; + private static final int FP32_SIGNIFICAND_MASK = 0x7fffff; + private static final int FP32_EXPONENT_BIAS = 127; + private static final int FP32_QNAN_MASK = 0x400000; + private static final int FP32_DENORMAL_MAGIC = 126 << 23; + private static final float FP32_DENORMAL_FLOAT = Float.intBitsToFloat(FP32_DENORMAL_MAGIC); + + /** + * Converts the specified half-precision float value into a single-precision float value. The following special + * cases are handled: If the input is NaN, the returned value is Float NaN. If the input is POSITIVE_INFINITY or + * NEGATIVE_INFINITY, the returned value is respectively Float POSITIVE_INFINITY or Float NEGATIVE_INFINITY. If the + * input is 0 (positive or negative), the returned value is +/-0.0f. Otherwise, the returned value is a normalized + * single-precision float value. + * + * @param b The half-precision float value to convert to single-precision + * @return A normalized single-precision float value + */ + public static float toFloat(short b) { + int bits = b & 0xffff; + int s = bits & SIGN_MASK; + int e = (bits >>> EXPONENT_SHIFT) & SHIFTED_EXPONENT_MASK; + int m = bits & SIGNIFICAND_MASK; + int outE = 0; + int outM = 0; + if (e == 0) { // Denormal or 0 + if (m != 0) { + // Convert denorm fp16 into normalized fp32 + float o = Float.intBitsToFloat(FP32_DENORMAL_MAGIC + m); + o -= FP32_DENORMAL_FLOAT; + return s == 0 ? o : -o; + } + } else { + outM = m << 13; + if (e == 0x1f) { // Infinite or NaN + outE = 0xff; + if (outM != 0) { // SNaNs are quieted + outM |= FP32_QNAN_MASK; + } + } else { + outE = e - EXPONENT_BIAS + FP32_EXPONENT_BIAS; + } + } + int out = (s << 16) | (outE << FP32_EXPONENT_SHIFT) | outM; + return Float.intBitsToFloat(out); + } + + /** + * Converts the specified single-precision float value into a half-precision float value. The following special + * cases are handled: + * + *

+ * If the input is NaN, the returned value is NaN. If the input is Float POSITIVE_INFINITY or Float + * NEGATIVE_INFINITY, the returned value is respectively POSITIVE_INFINITY or NEGATIVE_INFINITY. If the input is 0 + * (positive or negative), the returned value is POSITIVE_ZERO or NEGATIVE_ZERO. If the input is a less than + * MIN_VALUE, the returned value is flushed to POSITIVE_ZERO or NEGATIVE_ZERO. If the input is a less than + * MIN_NORMAL, the returned value is a denorm half-precision float. Otherwise, the returned value is rounded to the + * nearest representable half-precision float value. + * + * @param f The single-precision float value to convert to half-precision + * @return A half-precision float value + */ + public static short toFloat16(float f) { + int bits = Float.floatToIntBits(f); + int s = (bits >>> FP32_SIGN_SHIFT); + int e = (bits >>> FP32_EXPONENT_SHIFT) & FP32_SHIFTED_EXPONENT_MASK; + int m = bits & FP32_SIGNIFICAND_MASK; + int outE = 0; + int outM = 0; + if (e == 0xff) { // Infinite or NaN + outE = 0x1f; + outM = m != 0 ? 0x200 : 0; + } else { + e = e - FP32_EXPONENT_BIAS + EXPONENT_BIAS; + if (e >= 0x1f) { // Overflow + outE = 0x1f; + } else if (e <= 0) { // Underflow + if (e < -10) { + // The absolute fp32 value is less than MIN_VALUE, flush to +/-0 + } else { + // The fp32 value is a normalized float less than MIN_NORMAL, + // we convert to a denorm fp16 + m = m | 0x800000; + int shift = 14 - e; + outM = m >> shift; + int lowm = m & ((1 << shift) - 1); + int hway = 1 << (shift - 1); + // if above halfway or exactly halfway and outM is odd + if (lowm + (outM & 1) > hway) { + // Round to nearest even + // Can overflow into exponent bit, which surprisingly is OK. + // This increment relies on the +outM in the return statement below + outM++; + } + } + } else { + outE = e; + outM = m >> 13; + // if above halfway or exactly halfway and outM is odd + if ((m & 0x1fff) + (outM & 0x1) > 0x1000) { + // Round to nearest even + // Can overflow into exponent bit, which surprisingly is OK. + // This increment relies on the +outM in the return statement below + outM++; + } + } + } + // The outM is added here as the +1 increments for outM above can + // cause an overflow in the exponent bit which is OK. + return (short) ((s << SIGN_SHIFT) | (outE << EXPONENT_SHIFT) + outM); + } +} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReaderOptions.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReaderOptions.java index e00d9f3c6cd..da23127bde0 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReaderOptions.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/StreamReaderOptions.java @@ -6,34 +6,3 @@ import io.deephaven.extensions.barrage.ColumnConversionMode; import io.deephaven.util.QueryConstants; -public interface StreamReaderOptions { - /** - * @return whether we encode the validity buffer to express null values or {@link QueryConstants}'s NULL values. - */ - boolean useDeephavenNulls(); - - /** - * @return the conversion mode to use for object columns - */ - ColumnConversionMode columnConversionMode(); - - /** - * @return the ideal number of records to send per record batch - */ - int batchSize(); - - /** - * @return the maximum number of bytes that should be sent in a single message. - */ - int maxMessageSize(); - - /** - * Some Flight clients cannot handle modifications that have irregular column counts. These clients request that the - * server wrap all columns in a list to enable each column having a variable length. - * - * @return true if the columns should be wrapped in a list - */ - default boolean columnsAsList() { - return false; - } -} diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/TableToArrowConverter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/TableToArrowConverter.java index 948010a5f1d..9685bba7a93 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/TableToArrowConverter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/TableToArrowConverter.java @@ -5,8 +5,8 @@ import io.deephaven.engine.table.impl.BaseTable; import io.deephaven.extensions.barrage.BarragePerformanceLog; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; -import io.deephaven.extensions.barrage.BarrageStreamGeneratorImpl; +import io.deephaven.extensions.barrage.BarrageMessageWriter; +import io.deephaven.extensions.barrage.BarrageMessageWriterImpl; import io.grpc.stub.StreamObserver; import java.io.IOException; @@ -37,7 +37,7 @@ private void populateRecordBatches() { final BarragePerformanceLog.SnapshotMetricsHelper metrics = new BarragePerformanceLog.SnapshotMetricsHelper(); listener = new ArrowBuilderObserver(); - BarrageUtil.createAndSendSnapshot(new BarrageStreamGeneratorImpl.ArrowFactory(), table, null, null, + BarrageUtil.createAndSendSnapshot(new BarrageMessageWriterImpl.ArrowFactory(), table, null, null, false, DEFAULT_SNAPSHOT_DESER_OPTIONS, listener, metrics); } @@ -58,11 +58,11 @@ public byte[] next() { return listener.batchMessages.pop(); } - private static class ArrowBuilderObserver implements StreamObserver { + private static class ArrowBuilderObserver implements StreamObserver { final Deque batchMessages = new ArrayDeque<>(); @Override - public void onNext(final BarrageStreamGenerator.MessageView messageView) { + public void onNext(final BarrageMessageWriter.MessageView messageView) { try { messageView.forEachStream(inputStream -> { try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream()) { diff --git a/extensions/barrage/src/main/resources/io/deephaven/extensions/barrage/Barrage.gwt.xml b/extensions/barrage/src/main/resources/io/deephaven/extensions/barrage/Barrage.gwt.xml index a29af5b6ca8..b1b73cfb03a 100644 --- a/extensions/barrage/src/main/resources/io/deephaven/extensions/barrage/Barrage.gwt.xml +++ b/extensions/barrage/src/main/resources/io/deephaven/extensions/barrage/Barrage.gwt.xml @@ -3,12 +3,10 @@ - + - - - - + + diff --git a/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/BarrageStreamGeneratorTest.java b/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/BarrageStreamWriterTest.java similarity index 97% rename from extensions/barrage/src/test/java/io/deephaven/extensions/barrage/BarrageStreamGeneratorTest.java rename to extensions/barrage/src/test/java/io/deephaven/extensions/barrage/BarrageStreamWriterTest.java index 73be2b851af..42dd0bc5409 100644 --- a/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/BarrageStreamGeneratorTest.java +++ b/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/BarrageStreamWriterTest.java @@ -9,7 +9,7 @@ import java.io.IOException; -public class BarrageStreamGeneratorTest { +public class BarrageStreamWriterTest { @Test public void testDrainableStreamIsEmptied() throws IOException { diff --git a/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/chunk/BarrageColumnRoundTripTest.java b/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/chunk/BarrageColumnRoundTripTest.java index 505fd420ec8..82946b94547 100644 --- a/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/chunk/BarrageColumnRoundTripTest.java +++ b/extensions/barrage/src/test/java/io/deephaven/extensions/barrage/chunk/BarrageColumnRoundTripTest.java @@ -6,6 +6,7 @@ import com.google.common.io.LittleEndianDataInputStream; import com.google.protobuf.ByteString; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnDefinition; @@ -27,7 +28,6 @@ import io.deephaven.chunk.WritableShortChunk; import io.deephaven.extensions.barrage.util.BarrageUtil; import io.deephaven.extensions.barrage.util.ExposedByteArrayOutputStream; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.proto.flight.util.SchemaHelper; import io.deephaven.qst.type.Type; import io.deephaven.util.BooleanUtils; @@ -67,31 +67,30 @@ public class BarrageColumnRoundTripTest extends RefreshingTableTestCase { private static final BarrageSubscriptionOptions OPT_DEFAULT = BarrageSubscriptionOptions.builder() .build(); - private static final BarrageSubscriptionOptions[] options = new BarrageSubscriptionOptions[] { + private static final BarrageSubscriptionOptions[] OPTIONS = new BarrageSubscriptionOptions[] { OPT_DEFAULT_DH_NULLS, OPT_DEFAULT }; private static WritableChunk readChunk( - final StreamReaderOptions options, - final ChunkType chunkType, + final ChunkReader.Options options, final Class type, final Class componentType, final Field field, - final Iterator fieldNodeIter, + final Iterator fieldNodeIter, final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int offset, final int totalRows) throws IOException { - return DefaultChunkReadingFactory.INSTANCE - .getReader(options, typeInfo(chunkType, type, componentType, field)) + return DefaultChunkReaderFactory.INSTANCE + .newReader(typeInfo(type, componentType, field), options) .readChunk(fieldNodeIter, bufferInfoIter, is, outChunk, offset, totalRows); } public void testCharChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, char.class, (utO) -> { final WritableCharChunk chunk = utO.asWritableCharChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -117,7 +116,7 @@ public void testCharChunkSerialization() throws IOException { public void testBooleanChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, boolean.class, (utO) -> { final WritableByteChunk chunk = utO.asWritableByteChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -143,7 +142,7 @@ public void testBooleanChunkSerialization() throws IOException { public void testByteChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, byte.class, (utO) -> { final WritableByteChunk chunk = utO.asWritableByteChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -169,7 +168,7 @@ public void testByteChunkSerialization() throws IOException { public void testShortChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, short.class, (utO) -> { final WritableShortChunk chunk = utO.asWritableShortChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -195,7 +194,7 @@ public void testShortChunkSerialization() throws IOException { public void testIntChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, int.class, (utO) -> { final WritableIntChunk chunk = utO.asWritableIntChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -221,7 +220,7 @@ public void testIntChunkSerialization() throws IOException { public void testLongChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, long.class, (utO) -> { final WritableLongChunk chunk = utO.asWritableLongChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -247,7 +246,7 @@ public void testLongChunkSerialization() throws IOException { public void testFloatChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, float.class, (utO) -> { final WritableFloatChunk chunk = utO.asWritableFloatChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -273,7 +272,7 @@ public void testFloatChunkSerialization() throws IOException { public void testDoubleChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, double.class, (utO) -> { final WritableDoubleChunk chunk = utO.asWritableDoubleChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -299,7 +298,7 @@ public void testDoubleChunkSerialization() throws IOException { public void testInstantChunkSerialization() throws IOException { final Random random = new Random(0); - for (final BarrageSubscriptionOptions opts : options) { + for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, Instant.class, (utO) -> { final WritableObjectChunk chunk = utO.asWritableObjectChunk(); for (int i = 0; i < chunk.size(); ++i) { @@ -325,12 +324,12 @@ public void testStringSerialization() throws IOException { } public void testUniqueToStringSerializationDHNulls() throws IOException { - testRoundTripSerialization(OPT_DEFAULT_DH_NULLS, Unique.class, initObjectChunk(Unique::new), + testRoundTripSerialization(OPT_DEFAULT_DH_NULLS, Object.class, initObjectChunk(Unique::new), new ObjectToStringValidator<>()); } public void testUniqueToStringSerialization() throws IOException { - testRoundTripSerialization(OPT_DEFAULT, Unique.class, initObjectChunk(Unique::new), + testRoundTripSerialization(OPT_DEFAULT, Object.class, initObjectChunk(Unique::new), new ObjectToStringValidator<>()); } @@ -668,35 +667,43 @@ private static void testRoundTripSerialization( } else { chunkType = ChunkType.fromElementType(type); } + final Class readType; + if (type == Object.class) { + // noinspection unchecked + readType = (Class) String.class; + } else { + readType = type; + } + ByteString schemaBytes = BarrageUtil.schemaBytesFromTableDefinition( - TableDefinition.of(ColumnDefinition.of("col", Type.find(type))), Collections.emptyMap(), false); + TableDefinition.of(ColumnDefinition.of("col", Type.find(readType))), Collections.emptyMap(), false); Schema schema = SchemaHelper.flatbufSchema(schemaBytes.asReadOnlyByteBuffer()); Field field = schema.fields(0); final WritableChunk srcData = chunkType.makeWritableChunk(4096); initData.accept(srcData); - // The generator owns data; it is allowed to close it prematurely if the data needs to be converted to primitive + // The writer owns data; it is allowed to close it prematurely if the data needs to be converted to primitive final WritableChunk data = chunkType.makeWritableChunk(4096); data.copyFromChunk(srcData, 0, 0, srcData.size()); - try (SafeCloseable ignored = data; - ChunkInputStreamGenerator generator = DefaultChunkInputStreamGeneratorFactory.INSTANCE - .makeInputStreamGenerator(chunkType, type, type.getComponentType(), srcData, 0)) { + final ChunkWriter> writer = DefaultChunkWriterFactory.INSTANCE + .newWriter(ChunkReader.typeInfo(type, type.getComponentType(), field)); + try (SafeCloseable ignored = srcData; + final ChunkWriter.Context> context = writer.makeContext(data, 0)) { // full sub logic try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); - final ChunkInputStreamGenerator.DrainableColumn column = generator.getInputStream(options, null)) { - + final ChunkWriter.DrainableColumn column = writer.getInputStream(context, null, options)) { - final ArrayList fieldNodes = new ArrayList<>(); + final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes - .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); + .add(new ChunkWriter.FieldNodeInfo(numElements, nullCount))); final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); column.drainTo(baos); final DataInput dis = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); - try (final WritableChunk rtData = readChunk(options, chunkType, type, type.getComponentType(), + try (final WritableChunk rtData = readChunk(options, readType, readType.getComponentType(), field, fieldNodes.iterator(), bufferNodes.build().iterator(), dis, null, 0, 0)) { Assert.eq(data.size(), "data.size()", rtData.size(), "rtData.size()"); validator.assertExpected(data, rtData, null, 0); @@ -705,18 +712,18 @@ private static void testRoundTripSerialization( // empty subset try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); - final ChunkInputStreamGenerator.DrainableColumn column = - generator.getInputStream(options, RowSetFactory.empty())) { + final ChunkWriter.DrainableColumn column = + writer.getInputStream(context, RowSetFactory.empty(), options)) { - final ArrayList fieldNodes = new ArrayList<>(); + final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes - .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); + .add(new ChunkWriter.FieldNodeInfo(numElements, nullCount))); final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); column.drainTo(baos); final DataInput dis = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); - try (final WritableChunk rtData = readChunk(options, chunkType, type, type.getComponentType(), + try (final WritableChunk rtData = readChunk(options, readType, readType.getComponentType(), field, fieldNodes.iterator(), bufferNodes.build().iterator(), dis, null, 0, 0)) { Assert.eq(rtData.size(), "rtData.size()", 0); } @@ -732,18 +739,18 @@ private static void testRoundTripSerialization( } try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); final RowSet subset = builder.build(); - final ChunkInputStreamGenerator.DrainableColumn column = - generator.getInputStream(options, subset)) { + final ChunkWriter.DrainableColumn column = + writer.getInputStream(context, subset, options)) { - final ArrayList fieldNodes = new ArrayList<>(); + final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes - .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); + .add(new ChunkWriter.FieldNodeInfo(numElements, nullCount))); final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); column.drainTo(baos); final DataInput dis = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); - try (final WritableChunk rtData = readChunk(options, chunkType, type, type.getComponentType(), + try (final WritableChunk rtData = readChunk(options, readType, readType.getComponentType(), field, fieldNodes.iterator(), bufferNodes.build().iterator(), dis, null, 0, 0)) { Assert.eq(subset.intSize(), "subset.intSize()", rtData.size(), "rtData.size()"); validator.assertExpected(data, rtData, subset, 0); @@ -752,12 +759,12 @@ private static void testRoundTripSerialization( // test append to existing chunk logic try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); - final ChunkInputStreamGenerator.DrainableColumn column = - generator.getInputStream(options, null)) { + final ChunkWriter.DrainableColumn column = + writer.getInputStream(context, null, options)) { - final ArrayList fieldNodes = new ArrayList<>(); + final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes - .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); + .add(new ChunkWriter.FieldNodeInfo(numElements, nullCount))); final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); final long[] buffers = bufferNodes.build().toArray(); @@ -766,13 +773,13 @@ private static void testRoundTripSerialization( // first message DataInput dis = new LittleEndianDataInputStream( new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); - try (final WritableChunk rtData = readChunk(options, chunkType, type, type.getComponentType(), + try (final WritableChunk rtData = readChunk(options, readType, readType.getComponentType(), field, fieldNodes.iterator(), Arrays.stream(buffers).iterator(), dis, null, 0, data.size() * 2)) { // second message dis = new LittleEndianDataInputStream( new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); - final WritableChunk rtData2 = readChunk(options, chunkType, type, type.getComponentType(), + final WritableChunk rtData2 = readChunk(options, readType, readType.getComponentType(), field, fieldNodes.iterator(), Arrays.stream(buffers).iterator(), dis, rtData, data.size(), data.size() * 2); Assert.eq(rtData, "rtData", rtData2, "rtData2"); diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java index f34382297e0..964cf981255 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSnapshotImpl.java @@ -93,7 +93,7 @@ public class BarrageSnapshotImpl extends ReferenceCountedLivenessNode implements final MethodDescriptor snapshotDescriptor = getClientDoExchangeDescriptor(options, schema.computeWireChunkTypes(), schema.computeWireTypes(), schema.computeWireComponentTypes(), - new BarrageStreamReader(resultTable.getDeserializationTmConsumer())); + new BarrageMessageReaderImpl(resultTable.getDeserializationTmConsumer())); // We need to ensure that the DoExchange RPC does not get attached to the server RPC when this is being called // from a Deephaven server RPC thread. If we need to generalize this in the future, we may wrap this logic in a @@ -299,7 +299,7 @@ public MethodDescriptor getClientDoExchangeDescripto final ChunkType[] columnChunkTypes, final Class[] columnTypes, final Class[] componentTypes, - final StreamReader streamReader) { + final BarrageMessageReader streamReader) { final MethodDescriptor.Marshaller requestMarshaller = ProtoUtils.marshaller(FlightData.getDefaultInstance()); final MethodDescriptor descriptor = FlightServiceGrpc.getDoExchangeMethod(); @@ -320,14 +320,14 @@ private static class BarrageDataMarshaller implements MethodDescriptor.Marshalle private final ChunkType[] columnChunkTypes; private final Class[] columnTypes; private final Class[] componentTypes; - private final StreamReader streamReader; + private final BarrageMessageReader streamReader; public BarrageDataMarshaller( final BarrageSnapshotOptions options, final ChunkType[] columnChunkTypes, final Class[] columnTypes, final Class[] componentTypes, - final StreamReader streamReader) { + final BarrageMessageReader streamReader) { this.options = options; this.columnChunkTypes = columnChunkTypes; this.columnTypes = columnTypes; diff --git a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java index 26c0672e649..5ebf1bc8da4 100644 --- a/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java +++ b/java-client/barrage/src/main/java/io/deephaven/client/impl/BarrageSubscriptionImpl.java @@ -104,7 +104,7 @@ public class BarrageSubscriptionImpl extends ReferenceCountedLivenessNode implem final MethodDescriptor subscribeDescriptor = getClientDoExchangeDescriptor(options, schema.computeWireChunkTypes(), schema.computeWireTypes(), schema.computeWireComponentTypes(), - new BarrageStreamReader(resultTable.getDeserializationTmConsumer())); + new BarrageMessageReaderImpl(resultTable.getDeserializationTmConsumer())); // We need to ensure that the DoExchange RPC does not get attached to the server RPC when this is being called // from a Deephaven server RPC thread. If we need to generalize this in the future, we may wrap this logic in a @@ -346,7 +346,7 @@ public static MethodDescriptor getClientDoExchangeDe final ChunkType[] columnChunkTypes, final Class[] columnTypes, final Class[] componentTypes, - final StreamReader streamReader) { + final BarrageMessageReader streamReader) { final MethodDescriptor.Marshaller requestMarshaller = ProtoUtils.marshaller(FlightData.getDefaultInstance()); final MethodDescriptor descriptor = FlightServiceGrpc.getDoExchangeMethod(); @@ -367,14 +367,14 @@ public static class BarrageDataMarshaller implements MethodDescriptor.Marshaller private final ChunkType[] columnChunkTypes; private final Class[] columnTypes; private final Class[] componentTypes; - private final StreamReader streamReader; + private final BarrageMessageReader streamReader; public BarrageDataMarshaller( final BarrageSubscriptionOptions options, final ChunkType[] columnChunkTypes, final Class[] columnTypes, final Class[] componentTypes, - final StreamReader streamReader) { + final BarrageMessageReader streamReader) { this.options = options; this.columnChunkTypes = columnChunkTypes; this.columnTypes = columnTypes; diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateBarrageUtils.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateBarrageUtils.java index 35dadff92d0..fc0ea3f4127 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateBarrageUtils.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateBarrageUtils.java @@ -20,12 +20,9 @@ public class ReplicateBarrageUtils { public static void main(final String[] args) throws IOException { ReplicatePrimitiveCode.charToAllButBoolean("replicateBarrageUtils", - CHUNK_PACKAGE + "/CharChunkInputStreamGenerator.java"); - fixupChunkInputStreamGen(CHUNK_PACKAGE + "/IntChunkInputStreamGenerator.java", "Int"); - fixupChunkInputStreamGen(CHUNK_PACKAGE + "/LongChunkInputStreamGenerator.java", "Long"); - fixupChunkInputStreamGen(CHUNK_PACKAGE + "/DoubleChunkInputStreamGenerator.java", "Double"); + CHUNK_PACKAGE + "/CharChunkWriter.java"); - ReplicatePrimitiveCode.charToAllButBoolean("replicateBarrageUtils", + ReplicatePrimitiveCode.charToAllButBooleanAndFloats("replicateBarrageUtils", CHUNK_PACKAGE + "/CharChunkReader.java"); ReplicatePrimitiveCode.charToAllButBoolean("replicateBarrageUtils", @@ -50,7 +47,7 @@ private static void fixupVectorExpansionKernel(final @NotNull String path, final FileUtils.writeLines(file, lines); } - private static void fixupChunkInputStreamGen(final @NotNull String path, final @NotNull String type) + private static void fixupChunkWriterGen(final @NotNull String path, final @NotNull String type) throws IOException { final File file = new File(path); List lines = FileUtils.readLines(file, Charset.defaultCharset()); diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java index e7c66cec5b7..9e19531062e 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateSourcesAndChunks.java @@ -612,7 +612,8 @@ private static void replicateObjectChunks() throws IOException { "ObjectChunk EMPTY", "ObjectChunk EMPTY", - "static T\\[\\] makeArray", "static T[] makeArray"); + "static T\\[\\] makeArray", "static T[] makeArray", + "QueryConstants.NULL_OBJECT", "null"); lines = replaceRegion(lines, "makeArray", Arrays.asList( " public static T[] makeArray(int capacity) {", diff --git a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java index 275c0fd650e..0be257d563f 100644 --- a/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java +++ b/server/src/main/java/io/deephaven/server/arrow/ArrowFlightUtil.java @@ -25,7 +25,7 @@ import io.deephaven.engine.updategraph.UpdateGraph; import io.deephaven.extensions.barrage.BarragePerformanceLog; import io.deephaven.extensions.barrage.BarrageSnapshotOptions; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; import io.deephaven.extensions.barrage.table.BarrageTable; import io.deephaven.extensions.barrage.util.ArrowToTableConverter; @@ -61,14 +61,14 @@ public class ArrowFlightUtil { private static final Logger log = LoggerFactory.getLogger(ArrowFlightUtil.class); - private static class MessageViewAdapter implements StreamObserver { + private static class MessageViewAdapter implements StreamObserver { private final StreamObserver delegate; private MessageViewAdapter(StreamObserver delegate) { this.delegate = delegate; } - public void onNext(BarrageStreamGenerator.MessageView value) { + public void onNext(BarrageMessageWriter.MessageView value) { synchronized (delegate) { try { value.forEachStream(delegate::onNext); @@ -97,7 +97,7 @@ public void onCompleted() { Configuration.getInstance().getIntegerWithDefault("barrage.minUpdateInterval", 1000); public static void DoGetCustom( - final BarrageStreamGenerator.Factory streamGeneratorFactory, + final BarrageMessageWriter.Factory streamGeneratorFactory, final SessionState session, final TicketRouter ticketRouter, final Flight.Ticket request, @@ -135,7 +135,7 @@ public static void DoGetCustom( metrics.tableKey = BarragePerformanceLog.getKeyFor(table); // create an adapter for the response observer - final StreamObserver listener = + final StreamObserver listener = new MessageViewAdapter(observer); // push the schema to the listener @@ -357,14 +357,14 @@ public interface Factory { private final String myPrefix; private final SessionState session; - private final StreamObserver listener; + private final StreamObserver listener; private boolean isClosed = false; private boolean isFirstMsg = true; private final TicketRouter ticketRouter; - private final BarrageStreamGenerator.Factory streamGeneratorFactory; + private final BarrageMessageWriter.Factory streamGeneratorFactory; private final BarrageMessageProducer.Operation.Factory bmpOperationFactory; private final HierarchicalTableViewSubscription.Factory htvsFactory; private final BarrageMessageProducer.Adapter subscriptionOptAdapter; @@ -383,7 +383,7 @@ interface Handler extends Closeable { @AssistedInject public DoExchangeMarshaller( final TicketRouter ticketRouter, - final BarrageStreamGenerator.Factory streamGeneratorFactory, + final BarrageMessageWriter.Factory streamGeneratorFactory, final BarrageMessageProducer.Operation.Factory bmpOperationFactory, final HierarchicalTableViewSubscription.Factory htvsFactory, final BarrageMessageProducer.Adapter subscriptionOptAdapter, diff --git a/server/src/main/java/io/deephaven/server/arrow/ArrowModule.java b/server/src/main/java/io/deephaven/server/arrow/ArrowModule.java index 7f2b22aa464..17461410388 100644 --- a/server/src/main/java/io/deephaven/server/arrow/ArrowModule.java +++ b/server/src/main/java/io/deephaven/server/arrow/ArrowModule.java @@ -10,10 +10,10 @@ import io.deephaven.barrage.flatbuf.BarrageSnapshotRequest; import io.deephaven.barrage.flatbuf.BarrageSubscriptionRequest; import io.deephaven.extensions.barrage.BarrageSnapshotOptions; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; import io.deephaven.server.barrage.BarrageMessageProducer; -import io.deephaven.extensions.barrage.BarrageStreamGeneratorImpl; +import io.deephaven.extensions.barrage.BarrageMessageWriterImpl; import io.grpc.BindableService; import javax.inject.Singleton; @@ -30,8 +30,8 @@ public abstract class ArrowModule { @Provides @Singleton - static BarrageStreamGenerator.Factory bindStreamGenerator() { - return new BarrageStreamGeneratorImpl.Factory(); + static BarrageMessageWriter.Factory bindStreamGenerator() { + return new BarrageMessageWriterImpl.Factory(); } @Provides diff --git a/server/src/main/java/io/deephaven/server/arrow/FlightServiceGrpcImpl.java b/server/src/main/java/io/deephaven/server/arrow/FlightServiceGrpcImpl.java index f290dc75860..8e8603abfa8 100644 --- a/server/src/main/java/io/deephaven/server/arrow/FlightServiceGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/arrow/FlightServiceGrpcImpl.java @@ -13,7 +13,7 @@ import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; import io.deephaven.engine.table.impl.util.EngineMetrics; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.util.GrpcUtil; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; @@ -44,7 +44,7 @@ public class FlightServiceGrpcImpl extends FlightServiceGrpc.FlightServiceImplBa private static final Logger log = LoggerFactory.getLogger(FlightServiceGrpcImpl.class); private final ScheduledExecutorService executorService; - private final BarrageStreamGenerator.Factory streamGeneratorFactory; + private final BarrageMessageWriter.Factory streamGeneratorFactory; private final SessionService sessionService; private final SessionService.ErrorTransformer errorTransformer; private final TicketRouter ticketRouter; @@ -55,7 +55,7 @@ public class FlightServiceGrpcImpl extends FlightServiceGrpc.FlightServiceImplBa @Inject public FlightServiceGrpcImpl( @Nullable final ScheduledExecutorService executorService, - final BarrageStreamGenerator.Factory streamGeneratorFactory, + final BarrageMessageWriter.Factory streamGeneratorFactory, final SessionService sessionService, final SessionService.ErrorTransformer errorTransformer, final TicketRouter ticketRouter, diff --git a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java index e0420105d05..08f567486eb 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -10,6 +10,7 @@ import dagger.assisted.AssistedInject; import io.deephaven.base.formatters.FormatBitSet; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ResettableWritableObjectChunk; import io.deephaven.chunk.WritableChunk; @@ -31,15 +32,19 @@ import io.deephaven.engine.table.impl.util.UpdateCoalescer; import io.deephaven.engine.updategraph.*; import io.deephaven.engine.updategraph.impl.PeriodicUpdateGraph; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.BarragePerformanceLog; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; import io.deephaven.extensions.barrage.BarrageSubscriptionPerformanceLogger; +import io.deephaven.extensions.barrage.chunk.ChunkReader; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; +import io.deephaven.extensions.barrage.chunk.DefaultChunkWriterFactory; import io.deephaven.extensions.barrage.util.BarrageUtil; import io.deephaven.extensions.barrage.util.GrpcUtil; -import io.deephaven.extensions.barrage.util.StreamReader; +import io.deephaven.extensions.barrage.util.BarrageMessageReader; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; +import io.deephaven.proto.flight.util.SchemaHelper; import io.deephaven.server.session.SessionService; import io.deephaven.server.util.Scheduler; import io.deephaven.util.SafeCloseable; @@ -47,6 +52,9 @@ import io.deephaven.util.datastructures.LongSizedDataStructure; import io.grpc.StatusRuntimeException; import io.grpc.stub.StreamObserver; +import org.apache.arrow.flatbuf.Message; +import org.apache.arrow.flatbuf.Schema; +import org.apache.commons.lang3.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.HdrHistogram.Histogram; @@ -80,7 +88,7 @@ * It is possible to use this replication source to create subscriptions that propagate changes from one UGP to another * inside the same JVM. *

- * The client-side counterpart of this is the {@link StreamReader}. + * The client-side counterpart of this is the {@link BarrageMessageReader}. */ public class BarrageMessageProducer extends LivenessArtifact implements DynamicNode, NotificationStepReceiver { @@ -116,7 +124,7 @@ public interface Factory { private final Scheduler scheduler; private final SessionService.ErrorTransformer errorTransformer; - private final BarrageStreamGenerator.Factory streamGeneratorFactory; + private final BarrageMessageWriter.Factory streamGeneratorFactory; private final BaseTable parent; private final long updateIntervalMs; private final Runnable onGetSnapshot; @@ -125,7 +133,7 @@ public interface Factory { public Operation( final Scheduler scheduler, final SessionService.ErrorTransformer errorTransformer, - final BarrageStreamGenerator.Factory streamGeneratorFactory, + final BarrageMessageWriter.Factory streamGeneratorFactory, @Assisted final BaseTable parent, @Assisted final long updateIntervalMs) { this(scheduler, errorTransformer, streamGeneratorFactory, parent, updateIntervalMs, null); @@ -135,7 +143,7 @@ public Operation( public Operation( final Scheduler scheduler, final SessionService.ErrorTransformer errorTransformer, - final BarrageStreamGenerator.Factory streamGeneratorFactory, + final BarrageMessageWriter.Factory streamGeneratorFactory, final BaseTable parent, final long updateIntervalMs, @Nullable final Runnable onGetSnapshot) { @@ -196,7 +204,7 @@ public int hashCode() { private final String logPrefix; private final Scheduler scheduler; private final SessionService.ErrorTransformer errorTransformer; - private final BarrageStreamGenerator.Factory streamGeneratorFactory; + private final BarrageMessageWriter.Factory streamGeneratorFactory; private final BaseTable parent; private final long updateIntervalMs; @@ -213,6 +221,8 @@ public int hashCode() { /** the possibly reinterpretted source column */ private final ColumnSource[] sourceColumns; + /** the chunk writer per source column */ + private final ChunkWriter>[] chunkWriters; /** which source columns are object columns and thus need proactive garbage collection */ private final BitSet objectColumns = new BitSet(); /** internally, booleans are reinterpretted to bytes; however we need to be packed bitsets over Arrow */ @@ -305,7 +315,7 @@ public void close() { public BarrageMessageProducer( final Scheduler scheduler, final SessionService.ErrorTransformer errorTransformer, - final BarrageStreamGenerator.Factory streamGeneratorFactory, + final BarrageMessageWriter.Factory streamGeneratorFactory, final BaseTable parent, final long updateIntervalMs, final Runnable onGetSnapshot) { @@ -343,6 +353,22 @@ public BarrageMessageProducer( realColumnType = new Class[sourceColumns.length]; realColumnComponentType = new Class[sourceColumns.length]; + // lookup ChunkWriter mappings once, as they are constant for the lifetime of this producer + // noinspection unchecked + chunkWriters = (ChunkWriter>[]) new ChunkWriter[sourceColumns.length]; + + final MutableInt mi = new MutableInt(); + final Schema schema = SchemaHelper.flatbufSchema( + BarrageUtil.schemaBytesFromTable(parent).asReadOnlyByteBuffer()); + + parent.getColumnSourceMap().forEach((columnName, columnSource) -> { + int ii = mi.getAndIncrement(); + chunkWriters[ii] = DefaultChunkWriterFactory.INSTANCE.newWriter(ChunkReader.typeInfo( + ReinterpretUtils.maybeConvertToPrimitiveDataType(columnSource.getType()), + columnSource.getComponentType(), + schema.fields(ii))); + }); + // we start off with initial sizes of zero, because its quite possible no one will ever look at this table final int capacity = 0; @@ -410,9 +436,9 @@ public void setOnGetSnapshot(Runnable onGetSnapshot, boolean isPreSnap) { * job is run we clean up deleted subscriptions and rebuild any state that is used to filter recorded updates. * */ - private class Subscription { + private static class Subscription { final BarrageSubscriptionOptions options; - final StreamObserver listener; + final StreamObserver listener; final String logPrefix; RowSet viewport; // active viewport @@ -442,7 +468,7 @@ private class Subscription { WritableRowSet growingIncrementalViewport = null; // rows to be sent to the client from the current snapshot boolean isFirstSnapshot; // is this the first snapshot after a change to a subscriptions - private Subscription(final StreamObserver listener, + private Subscription(final StreamObserver listener, final BarrageSubscriptionOptions options, final BitSet subscribedColumns, @Nullable final RowSet initialViewport, @@ -470,7 +496,7 @@ public boolean isViewport() { * @param columnsToSubscribe The initial columns to subscribe to * @param initialViewport Initial viewport, to be owned by the subscription */ - public void addSubscription(final StreamObserver listener, + public void addSubscription(final StreamObserver listener, final BarrageSubscriptionOptions options, @Nullable final BitSet columnsToSubscribe, @Nullable final RowSet initialViewport, @@ -515,7 +541,7 @@ public void addSubscription(final StreamObserver listener, + private boolean findAndUpdateSubscription(final StreamObserver listener, final Consumer updateSubscription) { final Function, Boolean> findAndUpdate = (List subscriptions) -> { for (final Subscription sub : subscriptions) { @@ -543,14 +569,14 @@ private boolean findAndUpdateSubscription(final StreamObserver listener, + public boolean updateSubscription(final StreamObserver listener, @Nullable final RowSet newViewport, @Nullable final BitSet columnsToSubscribe) { // assume forward viewport when not specified return updateSubscription(listener, newViewport, columnsToSubscribe, false); } public boolean updateSubscription( - final StreamObserver listener, + final StreamObserver listener, @Nullable final RowSet newViewport, @Nullable final BitSet columnsToSubscribe, final boolean newReverseViewport) { @@ -582,7 +608,7 @@ public boolean updateSubscription( }); } - public void removeSubscription(final StreamObserver listener) { + public void removeSubscription(final StreamObserver listener) { findAndUpdateSubscription(listener, sub -> { sub.pendingDelete = true; if (log.isDebugEnabled()) { @@ -1457,8 +1483,8 @@ private void updateSubscriptionsSnapshotAndPropagate() { } if (snapshot != null) { - try (final BarrageStreamGenerator snapshotGenerator = - streamGeneratorFactory.newGenerator(snapshot, this::recordWriteMetrics)) { + try (final BarrageMessageWriter snapshotGenerator = + streamGeneratorFactory.newMessageWriter(snapshot, chunkWriters, this::recordWriteMetrics)) { if (log.isDebugEnabled()) { log.debug().append(logPrefix).append("Sending snapshot to ").append(activeSubscriptions.size()) .append(" subscriber(s).").endl(); @@ -1515,8 +1541,8 @@ private void updateSubscriptionsSnapshotAndPropagate() { private void propagateToSubscribers(final BarrageMessage message, final RowSet propRowSetForMessage) { // message is released via transfer to stream generator (as it must live until all view's are closed) - try (final BarrageStreamGenerator generator = streamGeneratorFactory.newGenerator( - message, this::recordWriteMetrics)) { + try (final BarrageMessageWriter bmw = streamGeneratorFactory.newMessageWriter( + message, chunkWriters, this::recordWriteMetrics)) { for (final Subscription subscription : activeSubscriptions) { if (subscription.pendingInitialSnapshot || subscription.pendingDelete) { continue; @@ -1539,7 +1565,7 @@ private void propagateToSubscribers(final BarrageMessage message, final RowSet p try (final RowSet clientView = vp != null ? propRowSetForMessage.subSetForPositions(vp, isReversed) : null) { - subscription.listener.onNext(generator.getSubView( + subscription.listener.onNext(bmw.getSubView( subscription.options, false, vp, subscription.reverseViewport, clientView, cols)); } catch (final Exception e) { try { @@ -1568,7 +1594,7 @@ private void clearObjectDeltaColumns(@NotNull final BitSet objectColumnsToClear) } private void propagateSnapshotForSubscription(final Subscription subscription, - final BarrageStreamGenerator snapshotGenerator) { + final BarrageMessageWriter snapshotGenerator) { boolean needsSnapshot = subscription.pendingInitialSnapshot; // This is a little confusing, but by the time we propagate, the `snapshotViewport`/`snapshotColumns` objects @@ -2306,7 +2332,6 @@ public synchronized void run() { scheduler.runAfterDelay(BarragePerformanceLog.CYCLE_DURATION_MILLIS, this); final BarrageSubscriptionPerformanceLogger logger = BarragePerformanceLog.getInstance().getSubscriptionLogger(); - // noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (logger) { flush(now, logger, enqueue, "EnqueueMillis"); flush(now, logger, aggregate, "AggregateMillis"); diff --git a/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableViewSubscription.java b/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableViewSubscription.java index a0e18a46ad6..9aacbd68cd9 100644 --- a/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableViewSubscription.java +++ b/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableViewSubscription.java @@ -8,6 +8,7 @@ import dagger.assisted.AssistedFactory; import dagger.assisted.AssistedInject; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.liveness.LivenessArtifact; @@ -21,6 +22,10 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.util.BarrageMessage; import io.deephaven.extensions.barrage.*; +import io.deephaven.extensions.barrage.chunk.ChunkReader; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; +import io.deephaven.extensions.barrage.chunk.DefaultChunkWriterFactory; +import io.deephaven.extensions.barrage.util.BarrageUtil; import io.deephaven.extensions.barrage.util.GrpcUtil; import io.deephaven.extensions.barrage.util.HierarchicalTableSchemaUtil; import io.deephaven.internal.log.LoggerFactory; @@ -38,6 +43,7 @@ import java.util.ArrayList; import java.util.BitSet; import java.util.List; +import java.util.Map; import java.util.function.Function; import java.util.function.LongConsumer; @@ -54,7 +60,7 @@ public class HierarchicalTableViewSubscription extends LivenessArtifact { public interface Factory { HierarchicalTableViewSubscription create( HierarchicalTableView view, - StreamObserver listener, + StreamObserver listener, BarrageSubscriptionOptions subscriptionOptions, long intervalMillis); } @@ -63,10 +69,10 @@ HierarchicalTableViewSubscription create( private final Scheduler scheduler; private final SessionService.ErrorTransformer errorTransformer; - private final BarrageStreamGenerator.Factory streamGeneratorFactory; + private final BarrageMessageWriter.Factory streamGeneratorFactory; private final HierarchicalTableView view; - private final StreamObserver listener; + private final StreamObserver listener; private final BarrageSubscriptionOptions subscriptionOptions; private final long intervalDurationNanos; @@ -105,9 +111,9 @@ private enum State { public HierarchicalTableViewSubscription( @NotNull final Scheduler scheduler, @NotNull final SessionService.ErrorTransformer errorTransformer, - @NotNull final BarrageStreamGenerator.Factory streamGeneratorFactory, + @NotNull final BarrageMessageWriter.Factory streamGeneratorFactory, @Assisted @NotNull final HierarchicalTableView view, - @Assisted @NotNull final StreamObserver listener, + @Assisted @NotNull final StreamObserver listener, @Assisted @NotNull final BarrageSubscriptionOptions subscriptionOptions, @Assisted final long intervalDurationMillis) { this.scheduler = scheduler; @@ -213,7 +219,9 @@ public void onUpdate(@NotNull final TableUpdate upstream) { } @Override - protected void onFailureInternal(@NotNull final Throwable originalException, @NotNull final Entry sourceEntry) { + protected void onFailureInternal( + @NotNull final Throwable originalException, + @Nullable final Entry sourceEntry) { if (state != State.Active) { return; } @@ -292,8 +300,8 @@ private void process() { } private static long buildAndSendSnapshot( - @NotNull final BarrageStreamGenerator.Factory streamGeneratorFactory, - @NotNull final StreamObserver listener, + @NotNull final BarrageMessageWriter.Factory streamGeneratorFactory, + @NotNull final StreamObserver listener, @NotNull final BarrageSubscriptionOptions subscriptionOptions, @NotNull final HierarchicalTableView view, @NotNull final LongConsumer snapshotNanosConsumer, @@ -336,6 +344,9 @@ private static long buildAndSendSnapshot( barrageMessage.shifted = RowSetShiftData.EMPTY; barrageMessage.addColumnData = new BarrageMessage.AddColumnData[numAvailableColumns]; + // noinspection unchecked + final ChunkWriter>[] chunkWriters = + (ChunkWriter>[]) new ChunkWriter[numAvailableColumns]; for (int ci = 0, di = 0; ci < numAvailableColumns; ++ci) { final BarrageMessage.AddColumnData addColumnData = new BarrageMessage.AddColumnData(); final ColumnDefinition columnDefinition = columnDefinitions.get(ci); @@ -351,12 +362,17 @@ private static long buildAndSendSnapshot( ReinterpretUtils.maybeConvertToPrimitiveChunkType(columnDefinition.getDataType()); } barrageMessage.addColumnData[ci] = addColumnData; + + chunkWriters[ci] = DefaultChunkWriterFactory.INSTANCE.newWriter(ChunkReader.typeInfo( + ReinterpretUtils.maybeConvertToPrimitiveDataType(columnDefinition.getDataType()), + columnDefinition.getComponentType(), + BarrageUtil.flatbufFieldFor(columnDefinition, Map.of()))); } barrageMessage.modColumnData = BarrageMessage.ZERO_MOD_COLUMNS; // 5. Send the BarrageMessage - final BarrageStreamGenerator streamGenerator = - streamGeneratorFactory.newGenerator(barrageMessage, writeMetricsConsumer); + final BarrageMessageWriter streamGenerator = + streamGeneratorFactory.newMessageWriter(barrageMessage, chunkWriters, writeMetricsConsumer); // Note that we're always specifying "isInitialSnapshot=true". This is to provoke the subscription view to // send the added rows on every snapshot, since (1) our added rows are flat, and thus cheap to send, and // (2) we're relying on added rows to signal the full expanded size to the client. @@ -477,7 +493,6 @@ public synchronized void run() { final BarrageSubscriptionPerformanceLogger logger = BarragePerformanceLog.getInstance().getSubscriptionLogger(); - // noinspection SynchronizationOnLocalVariableOrMethodParameter synchronized (logger) { flush(now, logger, snapshotNanos, "SnapshotMillis"); flush(now, logger, writeNanos, "WriteMillis"); diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageBlinkTableTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageBlinkTableTest.java index dcb7445077e..f58392d3adf 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageBlinkTableTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageBlinkTableTest.java @@ -27,10 +27,10 @@ import io.deephaven.engine.updategraph.UpdateSourceCombiner; import io.deephaven.engine.util.TableDiff; import io.deephaven.engine.util.TableTools; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; import io.deephaven.extensions.barrage.table.BarrageTable; -import io.deephaven.extensions.barrage.util.BarrageStreamReader; +import io.deephaven.extensions.barrage.util.BarrageMessageReaderImpl; import io.deephaven.extensions.barrage.util.BarrageUtil; import io.deephaven.proto.flight.util.SchemaHelper; import io.deephaven.server.arrow.ArrowModule; @@ -71,7 +71,7 @@ public class BarrageBlinkTableTest extends RefreshingTableTestCase { ArrowModule.class }) public interface TestComponent { - BarrageStreamGenerator.Factory getStreamGeneratorFactory(); + BarrageMessageWriter.Factory getStreamGeneratorFactory(); @Component.Builder interface Builder { @@ -191,7 +191,7 @@ private class RemoteClient { final BarrageDataMarshaller marshaller = new BarrageDataMarshaller( options, schema.computeWireChunkTypes(), schema.computeWireTypes(), schema.computeWireComponentTypes(), - new BarrageStreamReader(barrageTable.getDeserializationTmConsumer())); + new BarrageMessageReaderImpl(barrageTable.getDeserializationTmConsumer())); BarrageMessageRoundTripTest.DummyObserver dummyObserver = new BarrageMessageRoundTripTest.DummyObserver(marshaller, commandQueue); diff --git a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java index 77eab59b012..46620168cd3 100644 --- a/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java +++ b/server/src/test/java/io/deephaven/server/barrage/BarrageMessageRoundTripTest.java @@ -26,10 +26,10 @@ import io.deephaven.engine.updategraph.UpdateSourceCombiner; import io.deephaven.engine.util.TableDiff; import io.deephaven.engine.util.TableTools; -import io.deephaven.extensions.barrage.BarrageStreamGenerator; +import io.deephaven.extensions.barrage.BarrageMessageWriter; import io.deephaven.extensions.barrage.BarrageSubscriptionOptions; import io.deephaven.extensions.barrage.table.BarrageTable; -import io.deephaven.extensions.barrage.util.BarrageStreamReader; +import io.deephaven.extensions.barrage.util.BarrageMessageReaderImpl; import io.deephaven.extensions.barrage.util.BarrageUtil; import io.deephaven.extensions.barrage.util.ExposedByteArrayOutputStream; import io.deephaven.server.arrow.ArrowModule; @@ -74,7 +74,7 @@ public class BarrageMessageRoundTripTest extends RefreshingTableTestCase { ArrowModule.class }) public interface TestComponent { - BarrageStreamGenerator.Factory getStreamGeneratorFactory(); + BarrageMessageWriter.Factory getStreamGeneratorFactory(); @Component.Builder interface Builder { @@ -192,7 +192,7 @@ private class RemoteClient { final BarrageDataMarshaller marshaller = new BarrageDataMarshaller( options, barrageTable.getWireChunkTypes(), barrageTable.getWireTypes(), barrageTable.getWireComponentTypes(), - new BarrageStreamReader(barrageTable.getDeserializationTmConsumer())); + new BarrageMessageReaderImpl(barrageTable.getDeserializationTmConsumer())); this.dummyObserver = new DummyObserver(marshaller, commandQueue); if (viewport == null) { @@ -1408,7 +1408,7 @@ public void createTable() { } } - public static class DummyObserver implements StreamObserver { + public static class DummyObserver implements StreamObserver { volatile boolean completed = false; private final BarrageDataMarshaller marshaller; @@ -1420,7 +1420,7 @@ public static class DummyObserver implements StreamObserver { try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream()) { diff --git a/server/test-utils/src/main/java/io/deephaven/server/test/FlightMessageRoundTripTest.java b/server/test-utils/src/main/java/io/deephaven/server/test/FlightMessageRoundTripTest.java index 7b2a002f177..171006a20d3 100644 --- a/server/test-utils/src/main/java/io/deephaven/server/test/FlightMessageRoundTripTest.java +++ b/server/test-utils/src/main/java/io/deephaven/server/test/FlightMessageRoundTripTest.java @@ -1313,7 +1313,7 @@ public void testEmptyNestedPrimitiveArray() { final int numRows = 1; listWriter.startList(); listWriter.endList(); - listWriter.setValueCount(numRows); + listWriter.setValueCount(0); root.setRowCount(numRows); diff --git a/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebBarrageStreamReader.java b/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebBarrageMessageReader.java similarity index 94% rename from web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebBarrageStreamReader.java rename to web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebBarrageMessageReader.java index 6bf5196034e..aeb335c3364 100644 --- a/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebBarrageStreamReader.java +++ b/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebBarrageMessageReader.java @@ -11,10 +11,9 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; import io.deephaven.extensions.barrage.chunk.ChunkReader; import io.deephaven.extensions.barrage.util.FlatBufferIteratorAdapter; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; import io.deephaven.io.streams.ByteBufferInputStream; import io.deephaven.javascript.proto.dhinternal.arrow.flight.protocol.flight_pb.FlightData; import io.deephaven.util.datastructures.LongSizedDataStructure; @@ -41,7 +40,7 @@ * Consumes FlightData fields from Flight/Barrage producers and builds browser-compatible WebBarrageMessage payloads * that can be used to maintain table data. */ -public class WebBarrageStreamReader { +public class WebBarrageMessageReader { private static final int MAX_CHUNK_SIZE = Integer.MAX_VALUE - 8; // record progress in reading @@ -54,10 +53,10 @@ public class WebBarrageStreamReader { private WebBarrageMessage msg; private final WebChunkReaderFactory chunkReaderFactory = new WebChunkReaderFactory(); - private final List readers = new ArrayList<>(); + private final List>> readers = new ArrayList<>(); public WebBarrageMessage parseFrom( - final StreamReaderOptions options, + final ChunkReader.Options options, ChunkType[] columnChunkTypes, Class[] columnTypes, Class[] componentTypes, @@ -155,10 +154,8 @@ public WebBarrageMessage parseFrom( header.header(schema); for (int i = 0; i < schema.fieldsLength(); i++) { Field field = schema.fields(i); - ChunkReader chunkReader = chunkReaderFactory.getReader(options, - ChunkReader.typeInfo(columnChunkTypes[i], columnTypes[i], - componentTypes[i], field)); - readers.add(chunkReader); + readers.add(chunkReaderFactory.newReader( + ChunkReader.typeInfo(columnTypes[i], componentTypes[i], field), options)); } return null; } @@ -178,9 +175,9 @@ public WebBarrageMessage parseFrom( ByteBuffer body = TypedArrayHelper.wrap(flightData.getDataBody_asU8()); final LittleEndianDataInputStream ois = new LittleEndianDataInputStream(new ByteBufferInputStream(body)); - final Iterator fieldNodeIter = + final Iterator fieldNodeIter = new FlatBufferIteratorAdapter<>(batch.nodesLength(), - i -> new ChunkInputStreamGenerator.FieldNodeInfo(batch.nodes(i))); + i -> new ChunkWriter.FieldNodeInfo(batch.nodes(i))); final long[] bufferInfo = new long[batch.buffersLength()]; for (int bi = 0; bi < batch.buffersLength(); ++bi) { diff --git a/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebChunkReaderFactory.java b/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebChunkReaderFactory.java index d183aa491d3..5ba0081df1c 100644 --- a/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebChunkReaderFactory.java +++ b/web/client-api/src/main/java/io/deephaven/web/client/api/barrage/WebChunkReaderFactory.java @@ -4,6 +4,7 @@ package io.deephaven.web.client.api.barrage; import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableIntChunk; @@ -13,15 +14,16 @@ import io.deephaven.extensions.barrage.chunk.BooleanChunkReader; import io.deephaven.extensions.barrage.chunk.ByteChunkReader; import io.deephaven.extensions.barrage.chunk.CharChunkReader; -import io.deephaven.extensions.barrage.chunk.ChunkInputStreamGenerator; +import io.deephaven.extensions.barrage.chunk.ChunkWriter; import io.deephaven.extensions.barrage.chunk.ChunkReader; import io.deephaven.extensions.barrage.chunk.DoubleChunkReader; +import io.deephaven.extensions.barrage.chunk.ExpansionKernel; import io.deephaven.extensions.barrage.chunk.FloatChunkReader; import io.deephaven.extensions.barrage.chunk.IntChunkReader; +import io.deephaven.extensions.barrage.chunk.ListChunkReader; import io.deephaven.extensions.barrage.chunk.LongChunkReader; import io.deephaven.extensions.barrage.chunk.ShortChunkReader; -import io.deephaven.extensions.barrage.chunk.VarListChunkReader; -import io.deephaven.extensions.barrage.util.StreamReaderOptions; +import io.deephaven.extensions.barrage.chunk.array.ArrayExpansionKernel; import io.deephaven.util.BooleanUtils; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.web.client.api.BigDecimalWrapper; @@ -37,6 +39,7 @@ import org.apache.arrow.flatbuf.TimeUnit; import org.apache.arrow.flatbuf.Timestamp; import org.apache.arrow.flatbuf.Type; +import org.jetbrains.annotations.NotNull; import java.io.DataInput; import java.io.IOException; @@ -48,34 +51,37 @@ import java.util.PrimitiveIterator; /** - * Browser-compatible implementation of the ChunkReaderFactory, with a focus on reading from arrow types rather than - * successfully round-tripping to the Java server. + * Browser-compatible implementation of the {@link ChunkReader.Factory}, with a focus on reading from arrow types rather + * than successfully round-tripping to the Java server. *

* Includes some specific workarounds to handle nullability that will make more sense for the browser. */ public class WebChunkReaderFactory implements ChunkReader.Factory { + @SuppressWarnings("unchecked") @Override - public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReader.TypeInfo typeInfo) { + public > ChunkReader newReader( + @NotNull final ChunkReader.TypeInfo typeInfo, + @NotNull final ChunkReader.Options options) { switch (typeInfo.arrowField().typeType()) { case Type.Int: { Int t = new Int(); typeInfo.arrowField().type(t); switch (t.bitWidth()) { case 8: { - return new ByteChunkReader(options); + return (ChunkReader) new ByteChunkReader(options); } case 16: { if (t.isSigned()) { - return new ShortChunkReader(options); + return (ChunkReader) new ShortChunkReader(options); } - return new CharChunkReader(options); + return (ChunkReader) new CharChunkReader(options); } case 32: { - return new IntChunkReader(options); + return (ChunkReader) new IntChunkReader(options); } case 64: { if (t.isSigned()) { - return new LongChunkReader(options).transform(LongWrapper::of); + return (ChunkReader) new LongChunkReader(options).transform(LongWrapper::of); } throw new IllegalArgumentException("Unsigned 64bit integers not supported"); } @@ -87,11 +93,12 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade FloatingPoint t = new FloatingPoint(); typeInfo.arrowField().type(t); switch (t.precision()) { + case Precision.HALF: case Precision.SINGLE: { - return new FloatChunkReader(options); + return (ChunkReader) new FloatChunkReader(t.precision(), options); } case Precision.DOUBLE: { - return new DoubleChunkReader(options); + return (ChunkReader) new DoubleChunkReader(t.precision(), options); } default: throw new IllegalArgumentException( @@ -101,7 +108,7 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade case Type.Binary: { if (typeInfo.type() == BigIntegerWrapper.class) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> extractChunkFromInputStream( + totalRows) -> (T) extractChunkFromInputStream( is, fieldNodeIter, bufferInfoIter, @@ -110,7 +117,7 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade } if (typeInfo.type() == BigDecimalWrapper.class) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> extractChunkFromInputStream( + totalRows) -> (T) extractChunkFromInputStream( is, fieldNodeIter, bufferInfoIter, @@ -131,7 +138,7 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade } case Type.Utf8: { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> extractChunkFromInputStream(is, fieldNodeIter, + totalRows) -> (T) extractChunkFromInputStream(is, fieldNodeIter, bufferInfoIter, (buf, off, len) -> new String(buf, off, len, StandardCharsets.UTF_8), outChunk, outOffset, totalRows); } @@ -160,7 +167,7 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade chunk.set(outOffset + ii, BooleanUtils.byteAsBoolean(value)); } - return chunk; + return (T) chunk; } }; @@ -170,7 +177,8 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade typeInfo.arrowField().type(t); switch (t.unit()) { case DateUnit.MILLISECOND: - return new LongChunkReader(options).transform(millis -> DateWrapper.of(millis * 1000 * 1000)); + return (ChunkReader) new LongChunkReader(options) + .transform(millis -> DateWrapper.of(millis * 1000 * 1000)); default: throw new IllegalArgumentException("Unsupported Date unit: " + DateUnit.name(t.unit())); } @@ -180,7 +188,7 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade typeInfo.arrowField().type(t); switch (t.bitWidth()) { case TimeUnit.NANOSECOND: { - return new LongChunkReader(options).transform(DateWrapper::of); + return (ChunkReader) new LongChunkReader(options).transform(DateWrapper::of); } default: throw new IllegalArgumentException("Unsupported Time unit: " + TimeUnit.name(t.unit())); @@ -194,7 +202,7 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade if (!t.timezone().equals("UTC")) { throw new IllegalArgumentException("Unsupported tz " + t.timezone()); } - return new LongChunkReader(options).transform(DateWrapper::of); + return (ChunkReader) new LongChunkReader(options).transform(DateWrapper::of); } default: throw new IllegalArgumentException("Unsupported Timestamp unit: " + TimeUnit.name(t.unit())); @@ -203,14 +211,23 @@ public ChunkReader getReader(StreamReaderOptions options, int factor, ChunkReade case Type.List: { if (typeInfo.componentType() == byte.class) { return (fieldNodeIter, bufferInfoIter, is, outChunk, outOffset, - totalRows) -> extractChunkFromInputStream( + totalRows) -> (T) extractChunkFromInputStream( is, fieldNodeIter, bufferInfoIter, (buf, off, len) -> Arrays.copyOfRange(buf, off, off + len), outChunk, outOffset, totalRows); } - return new VarListChunkReader<>(options, typeInfo, this); + + final ChunkReader.TypeInfo componentTypeInfo = new ChunkReader.TypeInfo( + typeInfo.componentType(), + typeInfo.componentType().getComponentType(), + typeInfo.arrowField().children(0)); + final ChunkType chunkType = ListChunkReader.getChunkTypeFor(componentTypeInfo.type()); + final ExpansionKernel kernel = + ArrayExpansionKernel.makeExpansionKernel(chunkType, componentTypeInfo.type()); + final ChunkReader componentReader = newReader(componentTypeInfo, options); + return (ChunkReader) new ListChunkReader<>(ListChunkReader.Mode.DENSE, 0, kernel, componentReader); } default: throw new IllegalArgumentException("Unsupported type: " + Type.name(typeInfo.arrowField().typeType())); @@ -223,13 +240,13 @@ public interface Mapper { public static WritableObjectChunk extractChunkFromInputStream( final DataInput is, - final Iterator fieldNodeIter, + final Iterator fieldNodeIter, final PrimitiveIterator.OfLong bufferInfoIter, final Mapper mapper, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { - final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); + final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); final long validityBuffer = bufferInfoIter.nextLong(); final long offsetsBuffer = bufferInfoIter.nextLong(); final long payloadBuffer = bufferInfoIter.nextLong(); diff --git a/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/AbstractTableSubscription.java b/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/AbstractTableSubscription.java index 155dbb8271a..a0fb9f4734d 100644 --- a/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/AbstractTableSubscription.java +++ b/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/AbstractTableSubscription.java @@ -21,7 +21,7 @@ import io.deephaven.web.client.api.WorkerConnection; import io.deephaven.web.client.api.barrage.CompressedRangeSetReader; import io.deephaven.web.client.api.barrage.WebBarrageMessage; -import io.deephaven.web.client.api.barrage.WebBarrageStreamReader; +import io.deephaven.web.client.api.barrage.WebBarrageMessageReader; import io.deephaven.web.client.api.barrage.WebBarrageUtils; import io.deephaven.web.client.api.barrage.data.WebBarrageSubscription; import io.deephaven.web.client.api.barrage.stream.BiDiStream; @@ -466,7 +466,7 @@ private void onViewportChange(RangeSet serverViewport, BitSet serverColumns, boo } } - private final WebBarrageStreamReader reader = new WebBarrageStreamReader(); + private final WebBarrageMessageReader reader = new WebBarrageMessageReader(); private void onFlightData(FlightData data) { WebBarrageMessage message; diff --git a/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/TableViewportSubscription.java b/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/TableViewportSubscription.java index 203dc0c7890..3bedbfeccb8 100644 --- a/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/TableViewportSubscription.java +++ b/web/client-api/src/main/java/io/deephaven/web/client/api/subscription/TableViewportSubscription.java @@ -24,7 +24,7 @@ import io.deephaven.web.client.api.TableData; import io.deephaven.web.client.api.WorkerConnection; import io.deephaven.web.client.api.barrage.WebBarrageMessage; -import io.deephaven.web.client.api.barrage.WebBarrageStreamReader; +import io.deephaven.web.client.api.barrage.WebBarrageMessageReader; import io.deephaven.web.client.api.barrage.WebBarrageUtils; import io.deephaven.web.client.api.barrage.data.WebBarrageSubscription; import io.deephaven.web.client.api.barrage.stream.BiDiStream; @@ -362,7 +362,7 @@ public Promise snapshot(JsRangeSet rows, Column[] columns) { }, (rowsAdded, rowsRemoved, totalMods, shifted, modifiedColumnSet) -> { }); - WebBarrageStreamReader reader = new WebBarrageStreamReader(); + WebBarrageMessageReader reader = new WebBarrageMessageReader(); return new Promise<>((resolve, reject) -> { BiDiStream doExchange = connection().streamFactory().create(