diff --git a/R/rdeephaven/inst/tests/testthat/test_table_handle_wrapper.R b/R/rdeephaven/inst/tests/testthat/test_table_handle_wrapper.R index a16ee21c3e3..212cd6be0ed 100644 --- a/R/rdeephaven/inst/tests/testthat/test_table_handle_wrapper.R +++ b/R/rdeephaven/inst/tests/testthat/test_table_handle_wrapper.R @@ -56,6 +56,8 @@ test_that("is_static returns the correct value", { }) test_that("nrow returns the correct number of rows", { + skip() + data <- setup() expect_equal(nrow(data$th1), nrow(data$df1)) @@ -67,6 +69,8 @@ test_that("nrow returns the correct number of rows", { }) test_that("ncol returns the correct number of columns", { + skip() + data <- setup() expect_equal(ncol(data$th1), ncol(data$df1)) @@ -78,6 +82,8 @@ test_that("ncol returns the correct number of columns", { }) test_that("dim returns the correct dimension", { + skip() + data <- setup() expect_equal(dim(data$th1), dim(data$df1)) diff --git a/engine/api/src/main/java/io/deephaven/engine/table/Table.java b/engine/api/src/main/java/io/deephaven/engine/table/Table.java index ea85da7220d..6d606f464ac 100644 --- a/engine/api/src/main/java/io/deephaven/engine/table/Table.java +++ b/engine/api/src/main/java/io/deephaven/engine/table/Table.java @@ -218,7 +218,7 @@ public interface Table extends */ String BARRAGE_PERFORMANCE_KEY_ATTRIBUTE = "BarragePerformanceTableKey"; /** - * Set this to control the schema used for barrage serialization. + * Set an Apache Arrow POJO Schema to this attribute to control the column encoding used for barrage serialization. */ String BARRAGE_SCHEMA_ATTRIBUTE = "BarrageSchema"; diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/sized/SizedChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/sized/SizedChunk.java index ea2f177a259..df17f8b8b7e 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/sized/SizedChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/sized/SizedChunk.java @@ -37,7 +37,7 @@ public WritableChunk get() { /** * Ensure the underlying chunk has a capacity of at least {@code capacity}. - * + *

* The data and size of the returned chunk are undefined. * * @param capacity the minimum capacity for the chunk. @@ -56,9 +56,9 @@ public WritableChunk ensureCapacity(int capacity) { /** * Ensure the underlying chunk has a capacity of at least {@code capacity}. - * + *

* If the chunk has existing data, then it is copied to the new chunk. - * + *

* If the underlying chunk already exists, then the size of the chunk is the original size. If the chunk did not * exist, then the size of the returned chunk is zero. * diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java index 9642fba9bed..55567679c3b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BaseTable.java @@ -361,7 +361,13 @@ public enum CopyAttributeOperation { CopyAttributeOperation.Preview)); tempMap.put(BARRAGE_SCHEMA_ATTRIBUTE, EnumSet.of( - CopyAttributeOperation.Filter)); + CopyAttributeOperation.Filter, + CopyAttributeOperation.FirstBy, + CopyAttributeOperation.Flatten, + CopyAttributeOperation.LastBy, + CopyAttributeOperation.PartitionBy, + CopyAttributeOperation.Reverse, + CopyAttributeOperation.Sort)); attributeToCopySet = Collections.unmodifiableMap(tempMap); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java index 94544aeec38..88d05fdbf92 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/remote/ConstructSnapshot.java @@ -1411,12 +1411,7 @@ private static boolean snapshotColumnsParallel( final ExecutionContext executionContext, @NotNull final BarrageMessage snapshot) { final JobScheduler jobScheduler = new OperationInitializerJobScheduler(); - final CompletableFuture waitForParallelSnapshot = new CompletableFuture<>() { - @Override - public boolean completeExceptionally(Throwable ex) { - return super.completeExceptionally(ex); - } - }; + final CompletableFuture waitForParallelSnapshot = new CompletableFuture<>(); jobScheduler.iterateParallel( executionContext, logOutput -> logOutput.append("snapshotColumnsParallel"), diff --git a/engine/table/src/main/java/io/deephaven/engine/updategraph/impl/PeriodicUpdateGraph.java b/engine/table/src/main/java/io/deephaven/engine/updategraph/impl/PeriodicUpdateGraph.java index 32712aba8b1..8741ac010ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/updategraph/impl/PeriodicUpdateGraph.java +++ b/engine/table/src/main/java/io/deephaven/engine/updategraph/impl/PeriodicUpdateGraph.java @@ -100,8 +100,10 @@ public static Builder newBuilder(final String name) { public static final String DEFAULT_TARGET_CYCLE_DURATION_MILLIS_PROP = "PeriodicUpdateGraph.targetCycleDurationMillis"; - public static final int DEFAULT_TARGET_CYCLE_DURATION_MILLIS = - Configuration.getInstance().getIntegerWithDefault(DEFAULT_TARGET_CYCLE_DURATION_MILLIS_PROP, 1000); + + public static int getDefaultTargetCycleDurationMillis() { + return Configuration.getInstance().getIntegerWithDefault(DEFAULT_TARGET_CYCLE_DURATION_MILLIS_PROP, 1000); + } private final long defaultTargetCycleDurationMillis; private volatile long targetCycleDurationMillis; @@ -255,7 +257,7 @@ public boolean isCycleOnBudget(long cycleTimeNanos) { * Resets the run cycle time to the default target configured via the {@link Builder} setting. * * @implNote If the {@link Builder#targetCycleDurationMillis(long)} property is not set, this value defaults to - * {@link Builder#DEFAULT_TARGET_CYCLE_DURATION_MILLIS_PROP} which defaults to 1000ms. + * {@link #DEFAULT_TARGET_CYCLE_DURATION_MILLIS_PROP} which defaults to 1000ms. */ @SuppressWarnings("unused") public void resetTargetCycleDuration() { @@ -1169,7 +1171,7 @@ public static PeriodicUpdateGraph getInstance(final String name) { public static final class Builder { private final boolean allowUnitTestMode = Configuration.getInstance().getBooleanWithDefault(ALLOW_UNIT_TEST_MODE_PROP, false); - private long targetCycleDurationMillis = DEFAULT_TARGET_CYCLE_DURATION_MILLIS; + private long targetCycleDurationMillis = getDefaultTargetCycleDurationMillis(); private long minimumCycleDurationToLogNanos = DEFAULT_MINIMUM_CYCLE_DURATION_TO_LOG_NANOSECONDS; private String name; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java index 1bc55d5bca5..8f018f1cbed 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/BarrageMessageWriterImpl.java @@ -901,7 +901,7 @@ private void processBatches(Consumer visitor, final RecordBa } } - private static int findWriterForOffset(final ChunkWriter.Context[] chunks, final long offset) { + private static int findWriterForOffset(final ChunkWriter.Context[] chunks, final long offset) { // fast path for smaller updates if (chunks.length <= 1) { return 0; @@ -949,7 +949,7 @@ private int appendAddColumns(final RecordBatchMessageView view, final long start endPos = Long.MAX_VALUE; } if (addColumnData[0].chunks().length != 0) { - final ChunkWriter.Context writer = addColumnData[0].chunks()[chunkIdx]; + final ChunkWriter.Context writer = addColumnData[0].chunks()[chunkIdx]; endPos = Math.min(endPos, writer.getLastRowOffset()); shift = -writer.getRowOffset(); } @@ -981,9 +981,9 @@ private int appendAddColumns(final RecordBatchMessageView view, final long start // Add the drainable last as it is allowed to immediately close a row set the visitors need addStream.accept(drainableColumn); } else { - final ChunkWriter.Context> chunk = chunkListWriter.chunks()[chunkIdx]; + final ChunkWriter.Context context = chunkListWriter.chunks()[chunkIdx]; final ChunkWriter.DrainableColumn drainableColumn = chunkListWriter.writer().getInputStream( - chunk, + context, shift == 0 ? myAddedOffsets : adjustedOffsets, view.options()); drainableColumn.visitFieldNodes(fieldNodeListener); @@ -1008,8 +1008,8 @@ 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 ModColumnWriter mcd = modColumnData[ii]; - final ChunkWriter.Context[] chunks = mcd.chunkListWriter.chunks(); - if (chunks.length == 0) { + final ChunkWriter.Context[] contexts = mcd.chunkListWriter.chunks(); + if (contexts.length == 0) { continue; } @@ -1017,9 +1017,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 = findWriterForOffset(chunks, startPos); - if (chunkIdx < chunks.length - 1) { - maxLength = Math.min(maxLength, chunks[chunkIdx].getLastRowOffset() + 1 - startPos); + final int chunkIdx = findWriterForOffset(contexts, startPos); + if (chunkIdx < contexts.length - 1) { + maxLength = Math.min(maxLength, contexts[chunkIdx].getLastRowOffset() + 1 - startPos); } columnChunkIdx[ii] = chunkIdx; } @@ -1029,7 +1029,7 @@ private int appendModColumns(final RecordBatchMessageView view, final long start long numRows = 0; for (int ii = 0; ii < modColumnData.length; ++ii) { final ModColumnWriter mcd = modColumnData[ii]; - final ChunkWriter.Context> chunk = mcd.chunkListWriter.chunks().length == 0 + final ChunkWriter.Context context = mcd.chunkListWriter.chunks().length == 0 ? null : mcd.chunkListWriter.chunks()[columnChunkIdx[ii]]; @@ -1046,8 +1046,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 (chunk != null) { - endPos = Math.min(endPos, chunk.getLastRowOffset()); + if (context != null) { + endPos = Math.min(endPos, context.getLastRowOffset()); } } @@ -1065,7 +1065,7 @@ private int appendModColumns(final RecordBatchMessageView view, final long start numRows = Math.max(numRows, myModOffsets.size()); try { - final int numElements = chunk == null ? 0 : myModOffsets.intSize("BarrageStreamWriterImpl"); + final int numElements = context == 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 SingleElementListHeaderWriter listHeader = @@ -1084,11 +1084,11 @@ private int appendModColumns(final RecordBatchMessageView view, final long start // Add the drainable last as it is allowed to immediately close a row set the visitors need addStream.accept(drainableColumn); } else { - final long shift = -chunk.getRowOffset(); + final long shift = -context.getRowOffset(); // normalize to the chunk offsets try (final WritableRowSet adjustedOffsets = shift == 0 ? null : myModOffsets.shift(shift)) { final ChunkWriter.DrainableColumn drainableColumn = mcd.chunkListWriter.writer().getInputStream( - chunk, shift == 0 ? myModOffsets : adjustedOffsets, view.options()); + context, 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 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 index f8700fd57a3..2af375ae666 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListWriter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/ChunkListWriter.java @@ -12,31 +12,30 @@ import java.io.IOException; import java.util.List; -public class ChunkListWriter> implements SafeCloseable { - private final ChunkWriter writer; - private final ChunkWriter.Context[] contexts; +public class ChunkListWriter> implements SafeCloseable { + private final ChunkWriter writer; + private final ChunkWriter.Context[] contexts; public ChunkListWriter( - final ChunkWriter writer, - final List chunks) { + final ChunkWriter writer, + final List chunks) { this.writer = writer; - // noinspection unchecked - this.contexts = (ChunkWriter.Context[]) new ChunkWriter.Context[chunks.size()]; + this.contexts = new ChunkWriter.Context[chunks.size()]; long rowOffset = 0; for (int i = 0; i < chunks.size(); ++i) { - final SourceChunkType valuesChunk = chunks.get(i); + final SOURCE_CHUNK_TYPE valuesChunk = chunks.get(i); this.contexts[i] = writer.makeContext(valuesChunk, rowOffset); rowOffset += valuesChunk.size(); } } - public ChunkWriter writer() { + public ChunkWriter writer() { return writer; } - public ChunkWriter.Context[] chunks() { + public ChunkWriter.Context[] chunks() { return contexts; } @@ -46,8 +45,8 @@ public ChunkWriter.DrainableColumn empty(@NotNull final BarrageOptions options) @Override public void close() { - for (final ChunkWriter.Context context : contexts) { - context.decrementReferenceCount(); + for (final ChunkWriter.Context context : contexts) { + context.close(); } } } 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 index eef29da34e5..20e77a4873e 100644 --- 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 @@ -5,8 +5,13 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.util.datastructures.LongSizedDataStructure; +import org.jetbrains.annotations.NotNull; +import java.io.DataInput; +import java.io.IOException; import java.util.function.Function; import java.util.function.IntFunction; @@ -35,4 +40,25 @@ public static ChunkType getChunkTypeFor(final Class dest) { } return ChunkType.fromElementType(dest); } + + protected static void readValidityBuffer( + @NotNull final DataInput is, + final int numValidityLongs, + final long validityBufferLength, + @NotNull final WritableLongChunk isValid, + @NotNull final String DEBUG_NAME) throws IOException { + // 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 + } + } } 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 index 0a1e0f94046..f6853e75cb9 100644 --- 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 @@ -6,10 +6,12 @@ import io.deephaven.UncheckedDeephavenException; 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.BarrageOptions; +import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -21,28 +23,29 @@ public abstract class BaseChunkWriter> implements ChunkWriter { @FunctionalInterface - public interface IsRowNullProvider> { - boolean isRowNull(SOURCE_CHUNK_TYPE chunk, int idx); + public interface ChunkTransformer> { + Chunk transform(SOURCE_CHUNK_TYPE values); } public static final byte[] PADDING_BUFFER = new byte[8]; public static final int REMAINDER_MOD_8_MASK = 0x7; - protected final IsRowNullProvider isRowNullProvider; - protected final Supplier emptyChunkSupplier; + private final ChunkTransformer transformer; + private final Supplier emptyChunkSupplier; + /** the size of each element in bytes if fixed */ protected final int elementSize; /** whether we can use the wire value as a deephaven null for clients that support dh nulls */ protected final boolean dhNullable; /** whether the field is nullable */ - protected final boolean fieldNullable; + private final boolean fieldNullable; BaseChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, final int elementSize, final boolean dhNullable, final boolean fieldNullable) { - this.isRowNullProvider = isRowNullProvider; + this.transformer = transformer; this.emptyChunkSupplier = emptyChunkSupplier; this.elementSize = elementSize; this.dhNullable = dhNullable; @@ -51,25 +54,55 @@ public interface IsRowNullProvider> { @Override public final DrainableColumn getEmptyInputStream(final @NotNull BarrageOptions options) throws IOException { - try (Context context = makeContext(emptyChunkSupplier.get(), 0)) { + try (Context context = makeContext(emptyChunkSupplier.get(), 0)) { return getInputStream(context, null, options); } } @Override - public Context makeContext( - @NotNull final SOURCE_CHUNK_TYPE chunk, - final long rowOffset) { - return new Context<>(chunk, rowOffset); + public Context makeContext(@NotNull SOURCE_CHUNK_TYPE chunk, long rowOffset) { + if (transformer == null) { + return new Context(chunk, rowOffset); + } + try { + return new Context(transformer.transform(chunk), rowOffset); + } finally { + if (chunk instanceof PoolableChunk) { + ((PoolableChunk) chunk).close(); + } + } } - abstract class BaseChunkInputStream> extends DrainableColumn { + /** + * Compute the number of nulls in the subset. + * + * @param context the context for the chunk + * @param subset the subset of rows to consider + * @return the number of nulls in the subset + */ + protected abstract int computeNullCount( + @NotNull Context context, + @NotNull RowSequence subset); + + /** + * Update the validity buffer for the subset. + * + * @param context the context for the chunk + * @param subset the subset of rows to consider + * @param serContext the serialization context + */ + protected abstract void writeValidityBufferInternal( + @NotNull Context context, + @NotNull RowSequence subset, + @NotNull SerContext serContext); + + abstract class BaseChunkInputStream extends DrainableColumn { protected final CONTEXT_TYPE context; protected final RowSequence subset; protected final BarrageOptions options; - protected boolean read = false; - private int nullCount; + protected boolean hasBeenRead = false; + private final int nullCount; BaseChunkInputStream( @NotNull final CONTEXT_TYPE context, @@ -93,11 +126,7 @@ abstract class BaseChunkInputStream { - if (isRowNullProvider.isRowNull(context.getChunk(), (int) row)) { - ++nullCount; - } - }); + nullCount = computeNullCount(context, this.subset); } } @@ -120,7 +149,7 @@ protected int getRawSize() throws IOException { 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)); + return (hasBeenRead ? 0 : rawSize + (rawMod8 > 0 ? 8 - rawMod8 : 0)); } /** @@ -147,27 +176,8 @@ protected long writeValidityBuffer(final DataOutput dos) { return 0; } - final SerContext serContext = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(serContext.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException( - "Unexpected exception while draining data to OutputStream: ", e); - } - serContext.accumulator = 0; - serContext.count = 0; - }; - subset.forAllRowKeys(row -> { - if (!isRowNullProvider.isRowNull(context.getChunk(), (int) row)) { - serContext.accumulator |= 1L << serContext.count; - } - if (++serContext.count == 64) { - flush.run(); - } - }); - if (serContext.count > 0) { - flush.run(); + try (final SerContext serContext = new SerContext(dos)) { + writeValidityBufferInternal(context, subset, serContext); } return getValidityMapSerializationSizeFor(subset.intSize()); @@ -223,8 +233,43 @@ protected static int getNumLongsForBitPackOfSize(final int numElements) { return ((numElements + 63) / 64); } - protected static final class SerContext { - long accumulator = 0; - long count = 0; + protected static final class SerContext implements SafeCloseable { + private final DataOutput dos; + + private long accumulator = 0; + private long count = 0; + + public SerContext(@NotNull final DataOutput dos) { + this.dos = dos; + } + + public void setNextIsNull(boolean isNull) { + if (!isNull) { + accumulator |= 1L << count; + } + if (++count == 64) { + flush(); + } + } + + private void flush() { + if (count == 0) { + return; + } + + try { + dos.writeLong(accumulator); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + accumulator = 0; + count = 0; + } + + @Override + public void close() { + flush(); + } } } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BigDecimalChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BigDecimalChunkWriter.java new file mode 100644 index 00000000000..6f27a1c1c85 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BigDecimalChunkWriter.java @@ -0,0 +1,98 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.extensions.barrage.chunk; + +import io.deephaven.UncheckedDeephavenException; +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.util.mutable.MutableInt; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import java.io.DataOutput; +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.math.RoundingMode; +import java.util.function.Supplier; + +public class BigDecimalChunkWriter> + extends FixedWidthChunkWriter { + private static final String DEBUG_NAME = "BigDecimalWriter"; + + private final ArrowType.Decimal decimalType; + + public BigDecimalChunkWriter( + @Nullable final ChunkTransformer transformer, + final ArrowType.Decimal decimalType, + @NotNull final Supplier emptyChunkSupplier, + final int elementSize, + final boolean dhNullable, + final boolean fieldNullable) { + super(transformer, emptyChunkSupplier, elementSize, dhNullable, fieldNullable); + this.decimalType = decimalType; + } + + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asObjectChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asObjectChunk().isNull((int) row)); + }); + } + + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + 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(); + + subset.forAllRowKeys(rowKey -> { + try { + BigDecimal value = context.getChunk().asObjectChunk().get((int) rowKey); + + if (value.scale() != scale) { + value = value.setScale(decimalType.getScale(), RoundingMode.HALF_UP); + } + + byte[] bytes = value.unscaledValue().and(truncationMask).toByteArray(); + int numZeroBytes = byteWidth - bytes.length; + Assert.geqZero(numZeroBytes, "numZeroBytes"); + if (numZeroBytes > 0) { + dos.write(nullValue, 0, numZeroBytes); + } + dos.write(bytes); + + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } + }); + } +} 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 a2887ddda6d..bfafc36f2ba 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 @@ -95,19 +95,7 @@ public WritableByteChunk readChunk( final int numValidityLongs = (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final int numPayloadBytesNeeded = (int) ((nodeInfo.numElements + 7L) / 8L); if (payloadBuffer < numPayloadBytesNeeded) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java index 8ab636b44be..ebb2c38ffa0 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkWriter.java @@ -5,19 +5,19 @@ import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; -import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; +import io.deephaven.util.BooleanUtils; import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.util.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.io.OutputStream; -import static io.deephaven.util.QueryConstants.*; - public class BooleanChunkWriter extends BaseChunkWriter> { private static final String DEBUG_NAME = "BooleanChunkWriter"; private static final BooleanChunkWriter NULLABLE_IDENTITY_INSTANCE = new BooleanChunkWriter(true); @@ -28,20 +28,39 @@ public static BooleanChunkWriter getIdentity(boolean isNullable) { } private BooleanChunkWriter(final boolean isNullable) { - super(ByteChunk::isNull, ByteChunk::getEmptyChunk, 0, false, isNullable); + super(null, ByteChunk::getEmptyChunk, 0, false, isNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context> context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new BooleanChunkInputStream(context, subset, options); } - private class BooleanChunkInputStream extends BaseChunkInputStream>> { + @Override + protected int computeNullCount(@NotNull Context context, @NotNull RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asByteChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal(@NotNull Context context, @NotNull RowSequence subset, + @NotNull SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asByteChunk().isNull((int) row)); + }); + } + + private class BooleanChunkInputStream extends BaseChunkInputStream { private BooleanChunkInputStream( - @NotNull final Context> context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -73,41 +92,22 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer bytesWritten += writeValidityBuffer(dos); // write the payload buffer - final SerContext serContext = new SerContext(); - final Runnable flush = () -> { - try { - dos.writeLong(serContext.accumulator); - } catch (final IOException e) { - throw new UncheckedDeephavenException("Unexpected exception while draining data to OutputStream: ", - e); - } - serContext.accumulator = 0; - serContext.count = 0; - }; - - subset.forAllRowKeys(row -> { - final byte byteValue = context.getChunk().get((int) row); - if (byteValue != NULL_BYTE) { - serContext.accumulator |= (byteValue > 0 ? 1L : 0L) << serContext.count; - } - if (++serContext.count == 64) { - flush.run(); - } - }); - if (serContext.count > 0) { - flush.run(); + // we cheat and re-use validity buffer serialization code + try (final SerContext serContext = new SerContext(dos)) { + subset.forAllRowKeys(row -> serContext.setNextIsNull( + context.getChunk().asByteChunk().get((int) row) != BooleanUtils.TRUE_BOOLEAN_AS_BYTE)); } bytesWritten += getNumLongsForBitPackOfSize(subset.intSize(DEBUG_NAME)) * (long) Long.BYTES; 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 8e6f24d39f5..105f60c50ad 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 @@ -115,19 +115,7 @@ public WritableByteChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Byte.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index 3328e0617c4..626aaa14dcd 100644 --- 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 @@ -8,13 +8,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.ByteChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,42 +30,64 @@ public class ByteChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "ByteChunkWriter"; private static final ByteChunkWriter> NULLABLE_IDENTITY_INSTANCE = new ByteChunkWriter<>( - ByteChunk::isNull, ByteChunk::getEmptyChunk, ByteChunk::get, true); + null, ByteChunk::getEmptyChunk, true); private static final ByteChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new ByteChunkWriter<>( - ByteChunk::isNull, ByteChunk::getEmptyChunk, ByteChunk::get, false); - + null, ByteChunk::getEmptyChunk, false); public static ByteChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToByteTransformFunction> { - byte get(SourceChunkType sourceValues, int offset); + public static WritableByteChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableByteChunk output = WritableByteChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToByteTransformFunction transform; - public ByteChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToByteTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Byte.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Byte.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new ByteChunkInputStream(context, subset, options); } - private class ByteChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asByteChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asByteChunk().isNull((int) row)); + }); + } + + private class ByteChunkInputStream extends BaseChunkInputStream { private ByteChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -81,12 +108,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -95,7 +122,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeByte(transform.get(context.getChunk(), (int) row)); + dos.writeByte(context.getChunk().asByteChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); 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 e51e02b3c98..e1ac242bf91 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 @@ -111,19 +111,7 @@ public WritableCharChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Character.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index 09833d2dd55..a53f07a15be 100644 --- 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 @@ -4,13 +4,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.CharChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -21,42 +26,64 @@ public class CharChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "CharChunkWriter"; private static final CharChunkWriter> NULLABLE_IDENTITY_INSTANCE = new CharChunkWriter<>( - CharChunk::isNull, CharChunk::getEmptyChunk, CharChunk::get, true); + null, CharChunk::getEmptyChunk, true); private static final CharChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new CharChunkWriter<>( - CharChunk::isNull, CharChunk::getEmptyChunk, CharChunk::get, false); - + null, CharChunk::getEmptyChunk, false); public static CharChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToCharTransformFunction> { - char get(SourceChunkType sourceValues, int offset); + public static WritableCharChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableCharChunk output = WritableCharChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToCharTransformFunction transform; - public CharChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToCharTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Character.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Character.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new CharChunkInputStream(context, subset, options); } - private class CharChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asCharChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asCharChunk().isNull((int) row)); + }); + } + + private class CharChunkInputStream extends BaseChunkInputStream { private CharChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -77,12 +104,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -91,7 +118,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeChar(transform.get(context.getChunk(), (int) row)); + dos.writeChar(context.getChunk().asCharChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); 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 8ffa38fb445..b9c1584a0b9 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 @@ -18,7 +18,12 @@ import java.util.PrimitiveIterator; /** - * Consumes Flight/Barrage streams and transforms them into WritableChunks. + * The {@code ChunkReader} interface provides a mechanism for consuming Flight/Barrage streams and transforming them + * into {@link WritableChunk} instances for further processing. It facilitates efficient deserialization of columnar + * data, supporting various data types and logical structures. This interface is part of the Deephaven Barrage + * extensions for handling streamed data ingestion. + * + * @param The type of chunk being read, extending {@link WritableChunk} with {@link Values}. */ public interface ChunkReader> { 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 index 21efe610eaf..e918cae9f70 100644 --- 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 @@ -19,6 +19,14 @@ import java.io.IOException; +/** + * The {@code ChunkWriter} interface provides a mechanism for writing chunks of data into a structured format suitable + * for transmission in Apache Arrow's columnar format. It enables efficient handling of chunked data, including support + * for various data types and logical structures. This interface is part of the Deephaven Barrage extensions for + * efficient data streaming and processing. + * + * @param The type of chunk of source data, extending {@link Chunk} with {@link Values}. + */ public interface ChunkWriter> { /** @@ -44,7 +52,7 @@ > ChunkWriter newWriter( * @param rowOffset the offset into the logical message potentially spread over multiple chunks * @return a context for the given chunk */ - Context makeContext( + Context makeContext( @NotNull SOURCE_CHUNK_TYPE chunk, long rowOffset); @@ -57,7 +65,7 @@ Context makeContext( * @return a single-use DrainableColumn ready to be drained via grpc */ DrainableColumn getInputStream( - @NotNull Context context, + @NotNull Context context, @Nullable RowSet subset, @NotNull BarrageOptions options) throws IOException; @@ -70,8 +78,8 @@ DrainableColumn getInputStream( DrainableColumn getEmptyInputStream( @NotNull BarrageOptions options) throws IOException; - class Context> extends ReferenceCounted implements SafeCloseable { - private final T chunk; + class Context extends ReferenceCounted implements SafeCloseable { + private final Chunk chunk; private final long rowOffset; /** @@ -80,7 +88,7 @@ class Context> extends ReferenceCounted implements SafeC * @param chunk the chunk of data to be written * @param rowOffset the offset into the logical message potentially spread over multiple chunks */ - public Context(final T chunk, final long rowOffset) { + public Context(final Chunk chunk, final long rowOffset) { super(1); this.chunk = chunk; this.rowOffset = rowOffset; @@ -89,7 +97,7 @@ public Context(final T chunk, final long rowOffset) { /** * @return the chunk wrapped by this wrapper */ - T getChunk() { + Chunk getChunk() { return chunk; } @@ -122,7 +130,7 @@ public void close() { @Override protected void onReferenceCountAtZero() { if (chunk instanceof PoolableChunk) { - ((PoolableChunk) chunk).close(); + ((PoolableChunk) chunk).close(); } } } @@ -154,7 +162,7 @@ interface BufferListener { abstract class DrainableColumn extends DefensiveDrainable { /** - * Append the field nde to the flatbuffer payload via the supplied listener. + * Append the field node to the flatbuffer payload via the supplied listener. * * @param listener the listener to notify for each logical field node in this payload */ 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 index f17f64183df..3c52e581489 100644 --- 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 @@ -125,6 +125,7 @@ protected DefaultChunkReaderFactory() { register(ArrowType.ArrowTypeID.Int, BigDecimal.class, DefaultChunkReaderFactory::intToBigDecimal); register(ArrowType.ArrowTypeID.Bool, boolean.class, DefaultChunkReaderFactory::boolToBoolean); register(ArrowType.ArrowTypeID.Bool, Boolean.class, DefaultChunkReaderFactory::boolToBoolean); + // note that we hold boolean's in ByteChunks, so it's identical logic to read boolean as bytes. register(ArrowType.ArrowTypeID.Bool, byte.class, DefaultChunkReaderFactory::boolToBoolean); register(ArrowType.ArrowTypeID.FixedSizeBinary, byte[].class, DefaultChunkReaderFactory::fixedSizeBinaryToByteArray); @@ -206,9 +207,9 @@ public > ChunkReader newReaderPojo( int fixedSizeLength = 0; final ListChunkReader.Mode mode; if (typeId == ArrowType.ArrowTypeID.List) { - mode = ListChunkReader.Mode.DENSE; + mode = ListChunkReader.Mode.VARIABLE; } else if (typeId == ArrowType.ArrowTypeID.ListView) { - mode = ListChunkReader.Mode.SPARSE; + mode = ListChunkReader.Mode.VIEW; } else { mode = ListChunkReader.Mode.FIXED; fixedSizeLength = ((ArrowType.FixedSizeList) field.getType()).getListSize(); @@ -647,7 +648,7 @@ private static ChunkReader> decimalToBig } BigInteger unscaledValue = new BigInteger(value); - return unscaledValue.divide(BigInteger.ONE.pow(scale)); + return unscaledValue.divide(BigInteger.TEN.pow(scale)); }); } @@ -973,8 +974,7 @@ private static ChunkReader> intToChar( switch (bitWidth) { case 8: return CharChunkReader.transformTo(new ByteChunkReader(options), - (chunk, ii) -> maskIfOverflow(unsigned, Byte.BYTES, - QueryLanguageFunctionUtils.charCast(chunk.get(ii)))); + (chunk, ii) -> QueryLanguageFunctionUtils.charCast(chunk.get(ii))); case 16: if (unsigned) { return new CharChunkReader(options); @@ -1238,36 +1238,92 @@ 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; - } - + /** + * Applies a mask to handle overflow for unsigned values by constraining the value to the range that can be + * represented with the specified number of bytes. + *

+ * This method ensures that negative values (in the case of unsigned inputs) are masked to fit within the valid + * range for the given number of bytes, effectively wrapping them around to their equivalent unsigned + * representation. + *

+ * Special handling is included to preserve the value of null-equivalent constants and to skip masking for signed + * values. + * + * @param unsigned Whether the value should be treated as unsigned. + * @param numBytes The number of bytes to constrain the value to (e.g., 1 for byte, 2 for short). + * @param value The input value to potentially mask. + * @return The masked value if unsigned and overflow occurs; otherwise, the original value. + */ @SuppressWarnings("SameParameterValue") private static short maskIfOverflow(final boolean unsigned, final int numBytes, short value) { - if (unsigned && value != QueryConstants.NULL_SHORT && value < 0) { + if (unsigned && value != QueryConstants.NULL_SHORT) { value &= (short) ((1L << (numBytes * 8)) - 1); } return value; } + /** + * Applies a mask to handle overflow for unsigned values by constraining the value to the range that can be + * represented with the specified number of bytes. + *

+ * This method ensures that negative values (in the case of unsigned inputs) are masked to fit within the valid + * range for the given number of bytes, effectively wrapping them around to their equivalent unsigned + * representation. + *

+ * Special handling is included to preserve the value of null-equivalent constants and to skip masking for signed + * values. + * + * @param unsigned Whether the value should be treated as unsigned. + * @param numBytes The number of bytes to constrain the value to (e.g., 1 for byte, 2 for short). + * @param value The input value to potentially mask. + * @return The masked value if unsigned and overflow occurs; otherwise, the original value. + */ private static int maskIfOverflow(final boolean unsigned, final int numBytes, int value) { - if (unsigned && value != QueryConstants.NULL_INT && value < 0) { + if (unsigned && value != QueryConstants.NULL_INT) { value &= (int) ((1L << (numBytes * 8)) - 1); } return value; } + /** + * Applies a mask to handle overflow for unsigned values by constraining the value to the range that can be + * represented with the specified number of bytes. + *

+ * This method ensures that negative values (in the case of unsigned inputs) are masked to fit within the valid + * range for the given number of bytes, effectively wrapping them around to their equivalent unsigned + * representation. + *

+ * Special handling is included to preserve the value of null-equivalent constants and to skip masking for signed + * values. + * + * @param unsigned Whether the value should be treated as unsigned. + * @param numBytes The number of bytes to constrain the value to (e.g., 1 for byte, 2 for short). + * @param value The input value to potentially mask. + * @return The masked value if unsigned and overflow occurs; otherwise, the original value. + */ private static long maskIfOverflow(final boolean unsigned, final int numBytes, long value) { - if (unsigned && value != QueryConstants.NULL_LONG && value < 0) { + if (unsigned && value != QueryConstants.NULL_LONG) { value &= ((1L << (numBytes * 8)) - 1); } return value; } + /** + * Applies a mask to handle overflow for unsigned values by constraining the value to the range that can be + * represented with the specified number of bytes. + *

+ * This method ensures that negative values (in the case of unsigned inputs) are masked to fit within the valid + * range for the given number of bytes, effectively wrapping them around to their equivalent unsigned + * representation. + *

+ * Special handling is included to preserve the value of null-equivalent constants and to skip masking for signed + * values. + * + * @param unsigned Whether the value should be treated as unsigned. + * @param numBytes The number of bytes to constrain the value to (e.g., 1 for byte, 2 for short). + * @param value The input value to potentially mask. + * @return The masked value if unsigned and overflow occurs; otherwise, the original 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) { 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 index 77fca4cd630..24f08c37e83 100644 --- 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 @@ -3,7 +3,7 @@ // package io.deephaven.extensions.barrage.chunk; -import io.deephaven.base.verify.Assert; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; @@ -14,7 +14,15 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.WritableByteChunk; +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.rowset.RowSequence; import io.deephaven.engine.table.impl.lang.QueryLanguageFunctionUtils; import io.deephaven.engine.table.impl.preview.ArrayPreview; import io.deephaven.engine.table.impl.preview.DisplayWrapper; @@ -44,7 +52,6 @@ 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; @@ -81,16 +88,14 @@ ChunkWriter> make( new EnumMap<>(ArrowType.ArrowTypeID.class); 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, String.class, DefaultChunkWriterFactory::utf8FromObject); register(ArrowType.ArrowTypeID.Utf8, Object.class, DefaultChunkWriterFactory::utf8FromObject); - register(ArrowType.ArrowTypeID.Utf8, PyObject.class, DefaultChunkWriterFactory::utf8FromPyObject); + register(ArrowType.ArrowTypeID.Utf8, PyObject.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, @@ -101,7 +106,6 @@ protected DefaultChunkWriterFactory() { register(ArrowType.ArrowTypeID.Binary, BigInteger.class, DefaultChunkWriterFactory::binaryFromBigInt); register(ArrowType.ArrowTypeID.Binary, BigDecimal.class, DefaultChunkWriterFactory::binaryFromBigDecimal); register(ArrowType.ArrowTypeID.Binary, Schema.class, DefaultChunkWriterFactory::binaryFromSchema); - 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); @@ -126,16 +130,20 @@ protected DefaultChunkWriterFactory() { 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); } + /** + * Disables the default behavior of converting unknown types to their {@code toString()} representation. + *

+ * By default, the {@code DefaultChunkWriterFactory} will use an encoder that invokes {@code toString()} on any + * incoming types it does not recognize or have a specific handler for. This method disables that behavior, ensuring + * that unsupported types throw an exception when a writer cannot be provided. + */ public void disableToStringUnknownTypes() { toStringUnknownTypes = false; } @@ -214,9 +222,9 @@ public > ChunkWriter newWriterPojo( int fixedSizeLength = 0; final ListChunkReader.Mode mode; if (typeId == ArrowType.ArrowTypeID.List) { - mode = ListChunkReader.Mode.DENSE; + mode = ListChunkReader.Mode.VARIABLE; } else if (typeId == ArrowType.ArrowTypeID.ListView) { - mode = ListChunkReader.Mode.SPARSE; + mode = ListChunkReader.Mode.VIEW; } else { mode = ListChunkReader.Mode.FIXED; fixedSizeLength = ((ArrowType.FixedSizeList) field.getType()).getListSize(); @@ -321,44 +329,37 @@ protected void register( if (deephavenType == byte.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Byte.class, typeInfo -> new ByteChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + ByteChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } else if (deephavenType == short.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Short.class, typeInfo -> new ShortChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + ShortChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } else if (deephavenType == int.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Integer.class, typeInfo -> new IntChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + IntChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } else if (deephavenType == long.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Long.class, typeInfo -> new LongChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + LongChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } else if (deephavenType == char.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Character.class, typeInfo -> new CharChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + CharChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } else if (deephavenType == float.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Float.class, typeInfo -> new FloatChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + FloatChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } else if (deephavenType == double.class) { registeredFactories.computeIfAbsent(arrowType, k -> new HashMap<>()) .put(Double.class, typeInfo -> new DoubleChunkWriter>( - ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> TypeUtils.unbox(chunk.get(ii)), + DoubleChunkWriter::chunkUnboxer, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable())); } } @@ -378,55 +379,49 @@ private static long factorForTimeUnit(final TimeUnit unit) { } } - private static ChunkWriter> timestampFromLong( + private static ChunkWriter> timestampFromZonedDateTime( final BarrageTypeInfo typeInfo) { final ArrowType.Timestamp tsType = (ArrowType.Timestamp) typeInfo.arrowField().getType(); final long factor = factorForTimeUnit(tsType.getUnit()); // TODO (https://github.com/deephaven/deephaven-core/issues/5241): Inconsistent handling of ZonedDateTime // we do not know whether the incoming chunk source is a LongChunk or ObjectChunk - return new LongChunkWriter<>( - (Chunk source, int offset) -> { - if (source instanceof LongChunk) { - return source.asLongChunk().isNull(offset); - } - - return source.asObjectChunk().isNull(offset); - }, - LongChunk::getEmptyChunk, - (Chunk source, int offset) -> { - if (source instanceof LongChunk) { - final long value = source.asLongChunk().get(offset); - return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; - } + return new LongChunkWriter<>((Chunk source) -> { + if (source instanceof LongChunk && factor == 1) { + return source; + } - final ZonedDateTime value = source.asObjectChunk().get(offset); - return value == null ? QueryConstants.NULL_LONG : DateTimeUtils.epochNanos(value) / factor; - }, typeInfo.arrowField().isNullable()); + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + if (source instanceof LongChunk) { + final LongChunk longChunk = source.asLongChunk(); + for (int ii = 0; ii < source.size(); ++ii) { + final long value = longChunk.get(ii); + chunk.set(ii, longChunk.isNull(ii) ? value : value / factor); + } + } else { + for (int ii = 0; ii < source.size(); ++ii) { + final ZonedDateTime value = source.asObjectChunk().get(ii); + chunk.set(ii, value == null + ? QueryConstants.NULL_LONG + : DateTimeUtils.epochNanos(value) / factor); + } + } + return chunk; + }, LongChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); } - private static ChunkWriter> timestampFromInstant( + private static ChunkWriter> timestampFromInstant( final BarrageTypeInfo typeInfo) { final long factor = factorForTimeUnit(((ArrowType.Timestamp) typeInfo.arrowField().getType()).getUnit()); - return new LongChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (source, offset) -> { - final Instant value = source.get(offset); - return value == null ? QueryConstants.NULL_LONG : DateTimeUtils.epochNanos(value) / factor; - }, typeInfo.arrowField().isNullable()); - } - - private static ChunkWriter> timestampFromZonedDateTime( - final BarrageTypeInfo typeInfo) { - final ArrowType.Timestamp tsType = (ArrowType.Timestamp) typeInfo.arrowField().getType(); - final long factor = factorForTimeUnit(tsType.getUnit()); - return new LongChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (source, offset) -> { - final ZonedDateTime value = source.get(offset); - return value == null ? QueryConstants.NULL_LONG : DateTimeUtils.epochNanos(value) / factor; - }, typeInfo.arrowField().isNullable()); - } - - private static ChunkWriter> utf8FromString( - final BarrageTypeInfo typeInfo) { - return new VarBinaryChunkWriter<>(typeInfo.arrowField().isNullable(), - (out, item) -> out.write(item.getBytes(StandardCharsets.UTF_8))); + return new LongChunkWriter<>((LongChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final long value = source.get(ii); + chunk.set(ii, value == QueryConstants.NULL_LONG + ? QueryConstants.NULL_LONG + : value / factor); + } + return chunk; + }, LongChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); } private static ChunkWriter> utf8FromObject( @@ -435,30 +430,17 @@ private static ChunkWriter> utf8FromObject( (out, item) -> out.write(item.toString().getBytes(StandardCharsets.UTF_8))); } - private static ChunkWriter> utf8FromPyObject( - final BarrageTypeInfo typeInfo) { - return new VarBinaryChunkWriter<>(typeInfo.arrowField().isNullable(), - (out, item) -> out.write(item.toString().getBytes(StandardCharsets.UTF_8))); - } - - private static ChunkWriter> durationFromLong( - final BarrageTypeInfo typeInfo) { - final long factor = factorForTimeUnit(((ArrowType.Duration) typeInfo.arrowField().getType()).getUnit()); - return factor == 1 - ? LongChunkWriter.getIdentity(typeInfo.arrowField().isNullable()) - : new LongChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, (source, offset) -> { - final long value = source.get(offset); - return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; - }, typeInfo.arrowField().isNullable()); - } - private static ChunkWriter> durationFromDuration( final BarrageTypeInfo typeInfo) { final long factor = factorForTimeUnit(((ArrowType.Duration) typeInfo.arrowField().getType()).getUnit()); - return new LongChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (source, offset) -> { - final Duration value = source.get(offset); - return value == null ? QueryConstants.NULL_LONG : value.toNanos() / factor; - }, typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((ObjectChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final Duration value = source.get(ii); + chunk.set(ii, value == null ? QueryConstants.NULL_LONG : value.toNanos() / factor); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); } private static ChunkWriter> floatingPointFromFloat( @@ -466,20 +448,28 @@ private static ChunkWriter> floatingPointFromFloat( final ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) typeInfo.arrowField().getType(); switch (fpType.getPrecision()) { case HALF: - return new ShortChunkWriter<>(FloatChunk::isNull, FloatChunk::getEmptyChunk, (source, offset) -> { - final double value = source.get(offset); - return value == QueryConstants.NULL_FLOAT - ? QueryConstants.NULL_SHORT - : Float16.toFloat16((float) value); - }, typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((FloatChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final float value = source.get(ii); + chunk.set(ii, value == QueryConstants.NULL_FLOAT + ? QueryConstants.NULL_SHORT + : Float16.toFloat16(value)); + } + return chunk; + }, FloatChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case SINGLE: return FloatChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); case DOUBLE: - return new DoubleChunkWriter<>(FloatChunk::isNull, FloatChunk::getEmptyChunk, - (source, offset) -> QueryLanguageFunctionUtils.doubleCast(source.get(offset)), - typeInfo.arrowField().isNullable()); + return new DoubleChunkWriter<>((FloatChunk source) -> { + final WritableDoubleChunk chunk = WritableDoubleChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.doubleCast(source.get(ii))); + } + return chunk; + }, FloatChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected floating point precision: " + fpType.getPrecision()); @@ -491,17 +481,26 @@ private static ChunkWriter> floatingPointFromDouble( final ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) typeInfo.arrowField().getType(); switch (fpType.getPrecision()) { case HALF: - return new ShortChunkWriter<>(DoubleChunk::isNull, DoubleChunk::getEmptyChunk, (source, offset) -> { - final double value = source.get(offset); - return value == QueryConstants.NULL_DOUBLE - ? QueryConstants.NULL_SHORT - : Float16.toFloat16((float) value); - }, typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((DoubleChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final double value = source.get(ii); + chunk.set(ii, value == QueryConstants.NULL_DOUBLE + ? QueryConstants.NULL_SHORT + : Float16.toFloat16((float) value)); + } + return chunk; + }, DoubleChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case SINGLE: - return new FloatChunkWriter<>(DoubleChunk::isNull, DoubleChunk::getEmptyChunk, - (source, offset) -> QueryLanguageFunctionUtils.floatCast(source.get(offset)), - typeInfo.arrowField().isNullable()); + return new FloatChunkWriter<>((DoubleChunk source) -> { + final WritableFloatChunk chunk = WritableFloatChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.floatCast(source.get(ii))); + } + return chunk; + }, DoubleChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); + case DOUBLE: return DoubleChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); @@ -515,22 +514,34 @@ private static ChunkWriter> floatingPointFromBig final ArrowType.FloatingPoint fpType = (ArrowType.FloatingPoint) typeInfo.arrowField().getType(); switch (fpType.getPrecision()) { case HALF: - return new ShortChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (source, offset) -> { - final BigDecimal value = source.get(offset); - return value == null - ? QueryConstants.NULL_SHORT - : Float16.toFloat16(value.floatValue()); - }, typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((ObjectChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final BigDecimal value = source.get(ii); + chunk.set(ii, value == null + ? QueryConstants.NULL_SHORT + : Float16.toFloat16(value.floatValue())); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case SINGLE: - return new FloatChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (source, offset) -> QueryLanguageFunctionUtils.floatCast(source.get(offset)), - typeInfo.arrowField().isNullable()); + return new FloatChunkWriter<>((ObjectChunk source) -> { + final WritableFloatChunk chunk = WritableFloatChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.floatCast(source.get(ii))); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case DOUBLE: - return new DoubleChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (source, offset) -> QueryLanguageFunctionUtils.doubleCast(source.get(offset)), - typeInfo.arrowField().isNullable()); + return new DoubleChunkWriter<>((ObjectChunk source) -> { + final WritableDoubleChunk chunk = WritableDoubleChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.doubleCast(source.get(ii))); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected floating point precision: " + fpType.getPrecision()); @@ -570,32 +581,6 @@ private static ChunkWriter> binaryFromSchema( ArrowIpcUtil::serialize); } - private static ChunkWriter> timeFromLong( - final BarrageTypeInfo typeInfo) { - // See timeFromLocalTime's comment for more information on wire format. - final ArrowType.Time timeType = (ArrowType.Time) typeInfo.arrowField().getType(); - final int bitWidth = timeType.getBitWidth(); - final long factor = factorForTimeUnit(timeType.getUnit()); - switch (bitWidth) { - case 32: - return new IntChunkWriter<>(LongChunk::isNull, 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); - }, typeInfo.arrowField().isNullable()); - - case 64: - return new LongChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, (chunk, ii) -> { - long value = chunk.get(ii); - return value == QueryConstants.NULL_LONG ? QueryConstants.NULL_LONG : value / factor; - }, typeInfo.arrowField().isNullable()); - - default: - throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); - } - } - private static ChunkWriter> timeFromLocalTime( final BarrageTypeInfo typeInfo) { /* @@ -619,18 +604,24 @@ private static ChunkWriter> timeFromLocalTime( final long factor = factorForTimeUnit(timeType.getUnit()); switch (bitWidth) { case 32: - return new IntChunkWriter<>(ObjectChunk::isNull, 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); - }, typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ObjectChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final LocalTime value = source.get(ii); + chunk.set(ii, value == null ? QueryConstants.NULL_INT : (int) (value.toNanoOfDay() / factor)); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (chunk, ii) -> { - final LocalTime lt = chunk.get(ii); - return lt == null ? QueryConstants.NULL_LONG : lt.toNanoOfDay() / factor; - }, typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((ObjectChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final LocalTime value = source.get(ii); + chunk.set(ii, value == null ? QueryConstants.NULL_LONG : value.toNanoOfDay() / factor); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); @@ -641,236 +632,169 @@ private static ChunkWriter> decimalFromByte( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, ByteChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((ByteChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final byte value = source.get(ii); + if (value == QueryConstants.NULL_BYTE) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, ByteChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromChar( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, CharChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((CharChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final char value = source.get(ii); + if (value == QueryConstants.NULL_CHAR) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, CharChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromShort( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, ShortChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((ShortChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final short value = source.get(ii); + if (value == QueryConstants.NULL_SHORT) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, ShortChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromInt( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, IntChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((IntChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final int value = source.get(ii); + if (value == QueryConstants.NULL_INT) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, IntChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromLong( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, LongChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((LongChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final long value = source.get(ii); + if (value == QueryConstants.NULL_LONG) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, LongChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromBigInteger( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, ObjectChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (out, chunk, offset) -> { - BigInteger value = chunk.get(offset); - if (value == null) { - out.write(nullValue); - return; - } - writeBigDecimal(out, new BigDecimal(value), byteWidth, scale, truncationMask, nullValue); - }); + return new BigDecimalChunkWriter<>((ObjectChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final BigInteger value = source.get(ii); + if (value == null) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, new BigDecimal(value)); + } + return chunk; + }, decimalType, ObjectChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromFloat( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, FloatChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((FloatChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final float value = source.get(ii); + if (value == QueryConstants.NULL_FLOAT) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, FloatChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromDouble( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, DoubleChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); - }); + return new BigDecimalChunkWriter<>((DoubleChunk source) -> { + final WritableObjectChunk chunk = WritableObjectChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final double value = source.get(ii); + if (value == QueryConstants.NULL_DOUBLE) { + chunk.set(ii, null); + continue; + } + + chunk.set(ii, BigDecimal.valueOf(value)); + } + return chunk; + }, decimalType, DoubleChunk::getEmptyChunk, byteWidth, false, typeInfo.arrowField().isNullable()); } private static ChunkWriter> decimalFromBigDecimal( final BarrageTypeInfo typeInfo) { final ArrowType.Decimal decimalType = (ArrowType.Decimal) typeInfo.arrowField().getType(); 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::isNull, ObjectChunk::getEmptyChunk, byteWidth, false, - typeInfo.arrowField().isNullable(), - (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); + return new BigDecimalChunkWriter<>(null, decimalType, ObjectChunk::getEmptyChunk, byteWidth, false, + typeInfo.arrowField().isNullable()); } private static ChunkWriter> intFromByte( @@ -882,17 +806,29 @@ private static ChunkWriter> intFromByte( case 8: return ByteChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); case 16: - return new ShortChunkWriter<>(ByteChunk::isNull, ByteChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((ByteChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, ByteChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 32: - return new IntChunkWriter<>(ByteChunk::isNull, ByteChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ByteChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, ByteChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(ByteChunk::isNull, ByteChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((ByteChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, ByteChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -905,19 +841,31 @@ private static ChunkWriter> intFromShort( switch (bitWidth) { case 8: - return new ByteChunkWriter<>(ShortChunk::isNull, ShortChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((ShortChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, ShortChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: return ShortChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); case 32: - return new IntChunkWriter<>(ShortChunk::isNull, ShortChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ShortChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, ShortChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(ShortChunk::isNull, ShortChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((ShortChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, ShortChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -930,19 +878,31 @@ private static ChunkWriter> intFromInt( switch (bitWidth) { case 8: - return new ByteChunkWriter<>(IntChunk::isNull, IntChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((IntChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, IntChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: - return new ShortChunkWriter<>(IntChunk::isNull, IntChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((IntChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, IntChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 32: return IntChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(IntChunk::isNull, IntChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((IntChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, IntChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -955,17 +915,29 @@ private static ChunkWriter> intFromLong( switch (bitWidth) { case 8: - return new ByteChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((LongChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, LongChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: - return new ShortChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((LongChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, LongChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 32: - return new IntChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((LongChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, LongChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: return LongChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); default: @@ -973,28 +945,44 @@ private static ChunkWriter> intFromLong( } } - private static ChunkWriter> intFromObject( + private static ChunkWriter> intFromObject( final BarrageTypeInfo typeInfo) { final ArrowType.Int intType = (ArrowType.Int) typeInfo.arrowField().getType(); final int bitWidth = intType.getBitWidth(); switch (bitWidth) { case 8: - return new ByteChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((ObjectChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: - return new ShortChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((ObjectChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 32: - return new IntChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ObjectChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((ObjectChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -1008,25 +996,41 @@ private static ChunkWriter> intFromChar( switch (bitWidth) { case 8: - return new ByteChunkWriter<>(CharChunk::isNull, CharChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((CharChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, CharChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: if (unsigned) { return CharChunkWriter.getIdentity(typeInfo.arrowField().isNullable()); } else { - return new ShortChunkWriter<>(CharChunk::isNull, CharChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((CharChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, CharChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); } case 32: - return new IntChunkWriter<>(CharChunk::isNull, CharChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((CharChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, CharChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(CharChunk::isNull, CharChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((CharChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, CharChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -1039,21 +1043,37 @@ private static ChunkWriter> intFromFloat( switch (bitWidth) { case 8: - return new ByteChunkWriter<>(FloatChunk::isNull, FloatChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((FloatChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, FloatChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: - return new ShortChunkWriter<>(FloatChunk::isNull, FloatChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((FloatChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, FloatChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 32: - return new IntChunkWriter<>(FloatChunk::isNull, FloatChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((FloatChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, FloatChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(FloatChunk::isNull, FloatChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((FloatChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, FloatChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -1066,21 +1086,37 @@ private static ChunkWriter> intFromDouble( switch (bitWidth) { case 8: - return new ByteChunkWriter<>(DoubleChunk::isNull, DoubleChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.byteCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ByteChunkWriter<>((DoubleChunk source) -> { + final WritableByteChunk chunk = WritableByteChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.byteCast(source.get(ii))); + } + return chunk; + }, DoubleChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 16: - return new ShortChunkWriter<>(DoubleChunk::isNull, DoubleChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.shortCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new ShortChunkWriter<>((DoubleChunk source) -> { + final WritableShortChunk chunk = WritableShortChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.shortCast(source.get(ii))); + } + return chunk; + }, DoubleChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 32: - return new IntChunkWriter<>(DoubleChunk::isNull, DoubleChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((DoubleChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.intCast(source.get(ii))); + } + return chunk; + }, DoubleChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); case 64: - return new LongChunkWriter<>(DoubleChunk::isNull, DoubleChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.longCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); + return new LongChunkWriter<>((DoubleChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + chunk.set(ii, QueryLanguageFunctionUtils.longCast(source.get(ii))); + } + return chunk; + }, DoubleChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: throw new IllegalArgumentException("Unexpected bit width: " + bitWidth); } @@ -1095,63 +1131,29 @@ private static ChunkWriter> fixedSizeBinaryFromByteA final BarrageTypeInfo typeInfo) { final ArrowType.FixedSizeBinary fixedSizeBinary = (ArrowType.FixedSizeBinary) typeInfo.arrowField().getType(); final int elementWidth = fixedSizeBinary.getByteWidth(); - return new FixedWidthChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, elementWidth, false, - typeInfo.arrowField().isNullable(), - (out, chunk, offset) -> { - final byte[] data = chunk.get(offset); + return new FixedWidthObjectChunkWriter<>(elementWidth, false, + typeInfo.arrowField().isNullable()) { + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + subset.forAllRowKeys(row -> { + final byte[] data = context.getChunk().asObjectChunk().get((int) row); 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); + try { + dos.write(data); + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); + } }); - } - - private static ChunkWriter> dateFromInt( - final BarrageTypeInfo typeInfo) { - // see dateFromLocalDate's comment for more information on wire format - final ArrowType.Date dateType = (ArrowType.Date) typeInfo.arrowField().getType(); - switch (dateType.getUnit()) { - case DAY: - return new IntChunkWriter<>(IntChunk::isNull, IntChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); - - case MILLISECOND: - final long factor = Duration.ofDays(1).toMillis(); - return new LongChunkWriter<>(IntChunk::isNull, IntChunk::getEmptyChunk, (chunk, ii) -> { - final long value = QueryLanguageFunctionUtils.longCast(chunk.get(ii)); - return value == QueryConstants.NULL_LONG - ? QueryConstants.NULL_LONG - : (value * factor); - }, typeInfo.arrowField().isNullable()); - default: - throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); - } - } - - private static ChunkWriter> dateFromLong( - final BarrageTypeInfo typeInfo) { - // see dateFromLocalDate's comment for more information on wire format - final ArrowType.Date dateType = (ArrowType.Date) typeInfo.arrowField().getType(); - switch (dateType.getUnit()) { - case DAY: - return new IntChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, - (chunk, ii) -> QueryLanguageFunctionUtils.intCast(chunk.get(ii)), - typeInfo.arrowField().isNullable()); - - case MILLISECOND: - final long factor = Duration.ofDays(1).toMillis(); - return new LongChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, (chunk, ii) -> { - final long value = chunk.get(ii); - return value == QueryConstants.NULL_LONG - ? QueryConstants.NULL_LONG - : (value * factor); - }, typeInfo.arrowField().isNullable()); - default: - throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); - } + } + }; } private static ChunkWriter> dateFromLocalDate( @@ -1170,51 +1172,28 @@ private static ChunkWriter> dateFromLocalDate( final ArrowType.Date dateType = (ArrowType.Date) typeInfo.arrowField().getType(); switch (dateType.getUnit()) { case DAY: - return new IntChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (chunk, ii) -> { - final LocalDate value = chunk.get(ii); - return value == null ? QueryConstants.NULL_INT : (int) value.toEpochDay(); - }, typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ObjectChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final LocalDate value = source.get(ii); + chunk.set(ii, value == null ? QueryConstants.NULL_INT : (int) value.toEpochDay()); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); + case MILLISECOND: final long factor = Duration.ofDays(1).toMillis(); - return new LongChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (chunk, ii) -> { - final LocalDate value = chunk.get(ii); - return value == null ? QueryConstants.NULL_LONG : value.toEpochDay() * factor; - }, typeInfo.arrowField().isNullable()); - default: - throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); - } - } - - private static ChunkWriter> intervalFromDurationLong( - final BarrageTypeInfo typeInfo) { - // See intervalFromPeriod's comment for more information on wire format. - - final ArrowType.Interval intervalType = (ArrowType.Interval) typeInfo.arrowField().getType(); - 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: - final long nsPerDay = Duration.ofDays(1).toNanos(); - final long nsPerMs = Duration.ofMillis(1).toNanos(); - return new FixedWidthChunkWriter<>(LongChunk::isNull, LongChunk::getEmptyChunk, Integer.BYTES * 2, - false, typeInfo.arrowField().isNullable(), - (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 / nsPerDay)); - out.writeInt((int) ((value % nsPerDay) / nsPerMs)); - } - }); + return new LongChunkWriter<>((ObjectChunk source) -> { + final WritableLongChunk chunk = WritableLongChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final LocalDate value = source.get(ii); + chunk.set(ii, value == null ? QueryConstants.NULL_LONG : value.toEpochDay() * factor); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); default: - throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); + throw new IllegalArgumentException("Unexpected date unit: " + dateType.getUnit()); } } @@ -1231,19 +1210,30 @@ private static ChunkWriter> intervalFromDuration( case DAY_TIME: final long nsPerMs = Duration.ofMillis(1).toNanos(); - return new FixedWidthChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, Integer.BYTES * 2, - false, typeInfo.arrowField().isNullable(), - (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() / nsPerMs)); + return new FixedWidthObjectChunkWriter<>(Integer.BYTES * 2, false, typeInfo.arrowField().isNullable()) { + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + subset.forAllRowKeys(row -> { + final Duration value = context.getChunk().asObjectChunk().get((int) row); + try { + if (value == null) { + dos.writeInt(0); + dos.writeInt(0); + } else { + // days then millis + dos.writeInt((int) value.toDays()); + dos.writeInt((int) (value.getNano() / nsPerMs)); + } + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); } }); + } + }; default: throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); @@ -1277,39 +1267,71 @@ private static ChunkWriter> intervalFromPeriod( final ArrowType.Interval intervalType = (ArrowType.Interval) typeInfo.arrowField().getType(); switch (intervalType.getUnit()) { case YEAR_MONTH: - return new IntChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (chunk, ii) -> { - final Period value = chunk.get(ii); - return value == null ? QueryConstants.NULL_INT : value.getMonths() + value.getYears() * 12; - }, typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ObjectChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final Period value = source.get(ii); + chunk.set(ii, value == null + ? QueryConstants.NULL_INT + : value.getMonths() + value.getYears() * 12); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); + case DAY_TIME: - return new FixedWidthChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, Integer.BYTES * 2, - false, typeInfo.arrowField().isNullable(), - (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); + return new FixedWidthObjectChunkWriter<>(Integer.BYTES * 2, false, typeInfo.arrowField().isNullable()) { + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + subset.forAllRowKeys(row -> { + final Period value = context.getChunk().asObjectChunk().get((int) row); + try { + if (value == null) { + dos.writeInt(0); + dos.writeInt(0); + } else { + // days then millis + dos.writeInt(value.getDays()); + dos.writeInt(0); + } + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); } }); + } + }; + case MONTH_DAY_NANO: - return new FixedWidthChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - Integer.BYTES * 2 + Long.BYTES, false, typeInfo.arrowField().isNullable(), - (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); + return new FixedWidthObjectChunkWriter<>(Integer.BYTES * 2 + Long.BYTES, false, + typeInfo.arrowField().isNullable()) { + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + subset.forAllRowKeys(row -> { + final Period value = context.getChunk().asObjectChunk().get((int) row); + try { + if (value == null) { + dos.writeInt(0); + dos.writeInt(0); + dos.writeLong(0); + } else { + dos.writeInt(value.getMonths() + value.getYears() * 12); + dos.writeInt(value.getDays()); + dos.writeLong(0); + } + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); } }); + } + }; + default: throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); } @@ -1322,40 +1344,74 @@ private static ChunkWriter> intervalFromPeri final ArrowType.Interval intervalType = (ArrowType.Interval) typeInfo.arrowField().getType(); switch (intervalType.getUnit()) { case YEAR_MONTH: - return new IntChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, (chunk, ii) -> { - final Period value = chunk.get(ii).getPeriod(); - return value == null ? QueryConstants.NULL_INT : value.getMonths() + value.getYears() * 12; - }, typeInfo.arrowField().isNullable()); + return new IntChunkWriter<>((ObjectChunk source) -> { + final WritableIntChunk chunk = WritableIntChunk.makeWritableChunk(source.size()); + for (int ii = 0; ii < source.size(); ++ii) { + final PeriodDuration value = source.get(ii); + chunk.set(ii, value == null ? QueryConstants.NULL_INT + : value.getPeriod().getMonths() + value.getPeriod().getYears() * 12); + } + return chunk; + }, ObjectChunk::getEmptyChunk, typeInfo.arrowField().isNullable()); + case DAY_TIME: - return new FixedWidthChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, Integer.BYTES * 2, - false, typeInfo.arrowField().isNullable(), - (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()); + return new FixedWidthObjectChunkWriter(Integer.BYTES * 2, false, + typeInfo.arrowField().isNullable()) { + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + subset.forAllRowKeys(row -> { + final PeriodDuration value = + context.getChunk().asObjectChunk().get((int) row); + try { + if (value == null) { + dos.writeInt(0); + dos.writeInt(0); + } else { + // days then millis + dos.writeInt(value.getPeriod().getDays()); + dos.writeInt(value.getDuration().getNano()); + } + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); } }); + } + }; + case MONTH_DAY_NANO: - return new FixedWidthChunkWriter<>(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, - Integer.BYTES * 2 + Long.BYTES, false, typeInfo.arrowField().isNullable(), - (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()); + return new FixedWidthObjectChunkWriter<>(Integer.BYTES * 2 + Long.BYTES, false, + typeInfo.arrowField().isNullable()) { + @Override + protected void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset) { + subset.forAllRowKeys(row -> { + final PeriodDuration value = + context.getChunk().asObjectChunk().get((int) row); + try { + if (value == null) { + dos.writeInt(0); + dos.writeInt(0); + dos.writeLong(0); + } else { + final Period period = value.getPeriod(); + dos.writeInt(period.getMonths() + period.getYears() * 12); + dos.writeInt(period.getDays()); + dos.writeLong(value.getDuration().getNano()); + } + } catch (final IOException e) { + throw new UncheckedDeephavenException( + "Unexpected exception while draining data to OutputStream: ", e); } }); + } + }; + default: throw new IllegalArgumentException("Unexpected interval unit: " + intervalType.getUnit()); } 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 6be0ccb88e6..d91a85a88c2 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 @@ -78,19 +78,7 @@ public WritableDoubleChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Double.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index 60dbe84efe8..8b5dfb5672b 100644 --- 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 @@ -8,13 +8,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.DoubleChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,42 +30,64 @@ public class DoubleChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "DoubleChunkWriter"; private static final DoubleChunkWriter> NULLABLE_IDENTITY_INSTANCE = new DoubleChunkWriter<>( - DoubleChunk::isNull, DoubleChunk::getEmptyChunk, DoubleChunk::get, true); + null, DoubleChunk::getEmptyChunk, true); private static final DoubleChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new DoubleChunkWriter<>( - DoubleChunk::isNull, DoubleChunk::getEmptyChunk, DoubleChunk::get, false); - + null, DoubleChunk::getEmptyChunk, false); public static DoubleChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToDoubleTransformFunction> { - double get(SourceChunkType sourceValues, int offset); + public static WritableDoubleChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableDoubleChunk output = WritableDoubleChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToDoubleTransformFunction transform; - public DoubleChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToDoubleTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Double.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Double.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new DoubleChunkInputStream(context, subset, options); } - private class DoubleChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asDoubleChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asDoubleChunk().isNull((int) row)); + }); + } + + private class DoubleChunkInputStream extends BaseChunkInputStream { private DoubleChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -81,12 +108,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -95,7 +122,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeDouble(transform.get(context.getChunk(), (int) row)); + dos.writeDouble(context.getChunk().asDoubleChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); 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 index d528c59a28e..2eb0909b5c9 100644 --- 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 @@ -16,21 +16,41 @@ import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +/** + * The {@code ExpansionKernel} interface provides methods for transforming chunks containing complex or nested data + * structures into flattened representations, and vice versa. This enables efficient handling of columnar data in + * scenarios involving arrays, or {@link io.deephaven.vector.Vector vectors}, particularly within the Deephaven Barrage + * extensions for Flight/Barrage streams. + *

+ * An {@code ExpansionKernel} supports two primary operations: + *

+ * + * @param The type of data being processed by this kernel. + */ 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. + * Expands a chunk of nested or multi-element data ({@code T[]} or {@code Vector}) into a flattened chunk of + * elements ({@code T}), along with metadata describing the structure of the original data. *

- * If a non-zero {@code fixedSizeLength} is provided, then each row will be truncated or null-appended as - * appropriate to match the fixed size. + * The expansion involves unrolling arrays, or {@link io.deephaven.vector.Vector vectors}, or other multi-element + * types into a single contiguous chunk. The number of elements belonging to each original row is recorded in + * {@code offsetDest}, which allows reconstructing the original structure when needed. + *

+ * If a non-zero {@code fixedSizeLength} is provided, each row will be truncated or padded with nulls to match the + * fixed size. A negative {@code fixedSizeLength} will pick elements from the end of the array/vector. * - * @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 + * @param source The source chunk containing nested or multi-element data to expand. + * @param fixedSizeLength The fixed size for each row, or 0 for variable-length rows. A negative value will pick + * elements from the end. + * @param offsetDest The destination {@link WritableIntChunk} to store row offsets, or {@code null} if not needed. + * @param The attribute type of the source chunk. + * @return A flattened {@link WritableChunk} containing the expanded elements. */ WritableChunk expand( @NotNull ObjectChunk source, @@ -38,23 +58,28 @@ WritableChunk expand( @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. + * Contracts a flattened chunk of elements ({@code T}) back into a chunk of nested or multi-element data + * ({@code T[]} or {@code Vector}), using provided metadata (e.g., row offsets or lengths) to reconstruct the + * original structure. *

- * 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}. + * The contraction process supports multiple configurations: + *

* - * @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} + * @param source The source chunk containing flattened data to contract. + * @param sizePerElement The fixed size for each row, or 0 for variable-length rows. + * @param offsets An {@link IntChunk} describing row start positions, or {@code null}. + * @param lengths An {@link IntChunk} describing row lengths, or {@code null}. + * @param outChunk A reusable {@link WritableChunk} to store the contracted result, or {@code null}. + * @param outOffset The starting position for writing into {@code outChunk}. + * @param totalRows The total number of rows, or 0 if unknown. + * @param The attribute type of the source chunk. + * @return A {@link WritableObjectChunk} containing the reconstructed nested or multi-element data. */ WritableObjectChunk contract( @NotNull Chunk source, @@ -66,17 +91,20 @@ WritableObjectChunk contract( int totalRows); /** - * This computes the length of a given index depending on whether this is an Arrow FixedSizeList, List, or ListView. + * Computes the length of a row at the specified index, based on provided metadata (offsets and lengths). *

- * 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}. + * The size computation follows these rules: + *

    + *
  • If {@code offsets} is {@code null}, each row is assumed to have a fixed size of {@code sizePerOffset}.
  • + *
  • If {@code lengths} is {@code null}, the size is calculated from adjacent elements in {@code offsets}.
  • + *
  • If both {@code offsets} and {@code lengths} are provided, {@code lengths} determines the row size.
  • + *
* - * @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 + * @param ii The row index for which to compute the size. + * @param sizePerOffset The fixed size for each row, if applicable. + * @param offsets An {@link IntChunk} describing row start positions, or {@code null}. + * @param lengths An {@link IntChunk} describing row lengths, or {@code null}. + * @return The size of the row at the specified index. */ @FinalDefault default int computeSize( 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 index b823c0a60c1..913011e5c19 100644 --- 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 @@ -4,9 +4,9 @@ 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.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; @@ -18,38 +18,35 @@ import java.io.OutputStream; import java.util.function.Supplier; -public class FixedWidthChunkWriter> extends BaseChunkWriter { +public abstract 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 IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, final int elementSize, final boolean dhNullable, - final boolean fieldNullable, - final Appender appendItem) { - super(isRowNullProvider, emptyChunkSupplier, elementSize, dhNullable, fieldNullable); - this.appendItem = appendItem; + final boolean fieldNullable) { + super(transformer, emptyChunkSupplier, elementSize, dhNullable, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new FixedWidthChunkInputStream(context, subset, options); } - private class FixedWidthChunkInputStream extends BaseChunkInputStream> { + protected abstract void writePayload( + @NotNull final Context context, + @NotNull final DataOutput dos, + @NotNull final RowSequence subset); + + private class FixedWidthChunkInputStream extends BaseChunkInputStream { private FixedWidthChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -71,12 +68,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final DataOutput dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -86,14 +83,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { 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); - } - }); + writePayload(context, dos, subset); bytesWritten += elementSize * subset.size(); bytesWritten += writePadBuffer(dos, bytesWritten); diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthObjectChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthObjectChunkWriter.java new file mode 100644 index 00000000000..398b9bb4941 --- /dev/null +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthObjectChunkWriter.java @@ -0,0 +1,43 @@ +// +// 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.engine.rowset.RowSequence; +import io.deephaven.util.mutable.MutableInt; +import org.jetbrains.annotations.NotNull; + +public abstract class FixedWidthObjectChunkWriter extends FixedWidthChunkWriter> { + + public FixedWidthObjectChunkWriter( + final int elementSize, + final boolean dhNullable, + final boolean fieldNullable) { + super(null, ObjectChunk::getEmptyChunk, elementSize, dhNullable, fieldNullable); + } + + @Override + protected int computeNullCount( + @NotNull final BaseChunkWriter.Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asObjectChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final BaseChunkWriter.Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asObjectChunk().isNull((int) row)); + }); + } +} 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 48286dab83f..5008c2258ee 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 @@ -74,19 +74,7 @@ public WritableFloatChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Float.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index 7832beb98cf..dc2101994c9 100644 --- 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 @@ -8,13 +8,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.FloatChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,42 +30,64 @@ public class FloatChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "FloatChunkWriter"; private static final FloatChunkWriter> NULLABLE_IDENTITY_INSTANCE = new FloatChunkWriter<>( - FloatChunk::isNull, FloatChunk::getEmptyChunk, FloatChunk::get, true); + null, FloatChunk::getEmptyChunk, true); private static final FloatChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new FloatChunkWriter<>( - FloatChunk::isNull, FloatChunk::getEmptyChunk, FloatChunk::get, false); - + null, FloatChunk::getEmptyChunk, false); public static FloatChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToFloatTransformFunction> { - float get(SourceChunkType sourceValues, int offset); + public static WritableFloatChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableFloatChunk output = WritableFloatChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToFloatTransformFunction transform; - public FloatChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToFloatTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Float.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Float.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new FloatChunkInputStream(context, subset, options); } - private class FloatChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asFloatChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asFloatChunk().isNull((int) row)); + }); + } + + private class FloatChunkInputStream extends BaseChunkInputStream { private FloatChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -81,12 +108,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -95,7 +122,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeFloat(transform.get(context.getChunk(), (int) row)); + dos.writeFloat(context.getChunk().asFloatChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); 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 8ec0d0ddc29..a176096f495 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 @@ -115,19 +115,7 @@ public WritableIntChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Integer.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index 1e0ee6b9839..61aef4df3ae 100644 --- 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 @@ -8,13 +8,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.IntChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,42 +30,64 @@ public class IntChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "IntChunkWriter"; private static final IntChunkWriter> NULLABLE_IDENTITY_INSTANCE = new IntChunkWriter<>( - IntChunk::isNull, IntChunk::getEmptyChunk, IntChunk::get, true); + null, IntChunk::getEmptyChunk, true); private static final IntChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new IntChunkWriter<>( - IntChunk::isNull, IntChunk::getEmptyChunk, IntChunk::get, false); - + null, IntChunk::getEmptyChunk, false); public static IntChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToIntTransformFunction> { - int get(SourceChunkType sourceValues, int offset); + public static WritableIntChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableIntChunk output = WritableIntChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToIntTransformFunction transform; - public IntChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToIntTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Integer.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Integer.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new IntChunkInputStream(context, subset, options); } - private class IntChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asIntChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asIntChunk().isNull((int) row)); + }); + } + + private class IntChunkInputStream extends BaseChunkInputStream { private IntChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -81,12 +108,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -95,7 +122,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeInt(transform.get(context.getChunk(), (int) row)); + dos.writeInt(context.getChunk().asIntChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); 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 index 144a15bbc49..7d1356177cc 100644 --- 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 @@ -21,7 +21,7 @@ public class ListChunkReader extends BaseChunkReader> { public enum Mode { - FIXED, DENSE, SPARSE + FIXED, VARIABLE, VIEW } private static final String DEBUG_NAME = "ListChunkReader"; @@ -55,7 +55,7 @@ public WritableObjectChunk readChunk( // 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(); + final long lengthsBufferLength = mode != Mode.VIEW ? 0 : bufferInfoIter.nextLong(); if (nodeInfo.numElements == 0) { is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, @@ -68,28 +68,16 @@ public WritableObjectChunk readChunk( final WritableObjectChunk chunk; final int numValidityLongs = (nodeInfo.numElements + 63) / 64; - final int numOffsets = nodeInfo.numElements + (mode == Mode.DENSE ? 1 : 0); + final int numOffsets = nodeInfo.numElements + (mode == Mode.VARIABLE ? 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 + final WritableIntChunk lengths = mode != Mode.VIEW ? 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 - } + readValidityBuffer(is, numValidityLongs, validityBufferLength, isValid, DEBUG_NAME); // Read offsets: if (offsets != null) { @@ -125,6 +113,7 @@ public WritableObjectChunk readChunk( componentReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0)) { // noinspection unchecked chunk = (WritableObjectChunk) kernel.contract(inner, fixedSizeLength, offsets, lengths, + outChunk, outOffset, totalRows); long nextValid = 0; 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 index dfa2477a2bd..1df45065ea4 100644 --- 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 @@ -9,65 +9,90 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderSequential; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.util.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.io.OutputStream; -public class ListChunkWriter> - extends BaseChunkWriter> { +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; + private final ExpansionKernel kernel; + private final ChunkWriter componentWriter; public ListChunkWriter( final ListChunkReader.Mode mode, final int fixedSizeLength, - final ExpansionKernel kernel, - final ChunkWriter componentWriter, + final ExpansionKernel kernel, + final ChunkWriter componentWriter, final boolean fieldNullable) { - super(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, 0, false, fieldNullable); + super(null, ObjectChunk::getEmptyChunk, 0, false, fieldNullable); this.mode = mode; this.fixedSizeLength = fixedSizeLength; this.kernel = kernel; this.componentWriter = componentWriter; } + @Override + protected int computeNullCount( + @NotNull final ChunkWriter.Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asObjectChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final ChunkWriter.Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asObjectChunk().isNull((int) row)); + }); + } + @Override public Context makeContext( - @NotNull final ObjectChunk chunk, + @NotNull final ObjectChunk chunk, final long rowOffset) { return new Context(chunk, rowOffset); } - public final class Context extends ChunkWriter.Context> { + public final class Context extends ChunkWriter.Context { private final WritableIntChunk offsets; - private final ChunkWriter.Context innerContext; + private final ChunkWriter.Context innerContext; public Context( - @NotNull final ObjectChunk chunk, + @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); + int numOffsets = chunk.size() + (mode == ListChunkReader.Mode.VARIABLE ? 1 : 0); offsets = WritableIntChunk.makeWritableChunk(numOffsets); } // noinspection unchecked innerContext = componentWriter.makeContext( - (ComponentChunkType) kernel.expand(chunk, fixedSizeLength, offsets), 0); + (COMPONENT_CHUNK_TYPE) kernel.expand(chunk, fixedSizeLength, offsets), 0); } @Override @@ -80,9 +105,10 @@ protected void onReferenceCountAtZero() { @Override public DrainableColumn getInputStream( - @NotNull final ChunkWriter.Context> context, + @NotNull final ChunkWriter.Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { + // noinspection unchecked return new ListChunkInputStream((Context) context, subset, options); } @@ -144,7 +170,7 @@ public void visitBuffers(final BufferListener listener) { // offsets if (mode != ListChunkReader.Mode.FIXED) { long numOffsetBytes = Integer.BYTES * ((long) numElements); - if (numElements > 0 && mode == ListChunkReader.Mode.DENSE) { + if (numElements > 0 && mode == ListChunkReader.Mode.VARIABLE) { // we need an extra offset for the end of the last element numOffsetBytes += Integer.BYTES; } @@ -152,7 +178,7 @@ public void visitBuffers(final BufferListener listener) { } // lengths - if (mode == ListChunkReader.Mode.SPARSE) { + if (mode == ListChunkReader.Mode.VIEW) { long numLengthsBytes = Integer.BYTES * ((long) numElements); listener.noteLogicalBuffer(padBufferSize(numLengthsBytes)); } @@ -182,7 +208,7 @@ protected int getRawSize() throws IOException { // offsets if (mode != ListChunkReader.Mode.FIXED) { long numOffsetBytes = Integer.BYTES * ((long) numElements); - if (numElements > 0 && mode == ListChunkReader.Mode.DENSE) { + if (numElements > 0 && mode == ListChunkReader.Mode.VARIABLE) { // we need an extra offset for the end of the last element numOffsetBytes += Integer.BYTES; } @@ -190,7 +216,7 @@ protected int getRawSize() throws IOException { } // lengths - if (mode == ListChunkReader.Mode.SPARSE) { + if (mode == ListChunkReader.Mode.VIEW) { long numLengthsBytes = Integer.BYTES * ((long) numElements); size += padBufferSize(numLengthsBytes); } @@ -204,18 +230,18 @@ protected int getRawSize() throws IOException { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } - read = true; + hasBeenRead = 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) { + if (mode == ListChunkReader.Mode.VARIABLE) { // write down only offset (+1) buffer final WritableIntChunk offsetsToUse = myOffsets == null ? context.offsets : myOffsets; for (int i = 0; i < offsetsToUse.size(); ++i) { @@ -223,7 +249,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { } bytesWritten += ((long) offsetsToUse.size()) * Integer.BYTES; bytesWritten += writePadBuffer(dos, bytesWritten); - } else if (mode == ListChunkReader.Mode.SPARSE) { + } else if (mode == ListChunkReader.Mode.VIEW) { // write down offset buffer final WritableIntChunk offsetsToUse = myOffsets == null ? context.offsets : myOffsets; 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 8663d530da9..85706b4ed3a 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 @@ -115,19 +115,7 @@ public WritableLongChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Long.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index 63ec4638837..9d6f49899d5 100644 --- 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 @@ -8,13 +8,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.LongChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,42 +30,64 @@ public class LongChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "LongChunkWriter"; private static final LongChunkWriter> NULLABLE_IDENTITY_INSTANCE = new LongChunkWriter<>( - LongChunk::isNull, LongChunk::getEmptyChunk, LongChunk::get, true); + null, LongChunk::getEmptyChunk, true); private static final LongChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new LongChunkWriter<>( - LongChunk::isNull, LongChunk::getEmptyChunk, LongChunk::get, false); - + null, LongChunk::getEmptyChunk, false); public static LongChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToLongTransformFunction> { - long get(SourceChunkType sourceValues, int offset); + public static WritableLongChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableLongChunk output = WritableLongChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToLongTransformFunction transform; - public LongChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToLongTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Long.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Long.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new LongChunkInputStream(context, subset, options); } - private class LongChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asLongChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asLongChunk().isNull((int) row)); + }); + } + + private class LongChunkInputStream extends BaseChunkInputStream { private LongChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -81,12 +108,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -95,7 +122,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeLong(transform.get(context.getChunk(), (int) row)); + dos.writeLong(context.getChunk().asLongChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkReader.java index 7e3e939ec75..56cbcd9b3c6 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkReader.java @@ -43,14 +43,16 @@ public WritableObjectChunk readChunk( final int outOffset, final int totalRows) throws IOException { final ChunkWriter.FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final ChunkWriter.FieldNodeInfo innerInfo = fieldNodeIter.next(); + // an arrow map is represented as a List>>; the struct is superfluous, but we must + // consume the field node anyway + final ChunkWriter.FieldNodeInfo structInfo = fieldNodeIter.next(); final long validityBufferLength = bufferInfoIter.nextLong(); final long offsetsBufferLength = bufferInfoIter.nextLong(); - final long structValiadityBufferLength = bufferInfoIter.nextLong(); + final long structValidityBufferLength = bufferInfoIter.nextLong(); if (nodeInfo.numElements == 0) { is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, - validityBufferLength + offsetsBufferLength + structValiadityBufferLength)); + validityBufferLength + offsetsBufferLength + structValidityBufferLength)); try (final WritableChunk ignored = keyReader.readChunk(fieldNodeIter, bufferInfoIter, is, null, 0, 0); final WritableChunk ignored2 = @@ -65,19 +67,7 @@ public WritableObjectChunk readChunk( try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs); final WritableIntChunk offsets = WritableIntChunk.makeWritableChunk(numOffsets)) { - // 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 - } + readValidityBuffer(is, numValidityLongs, validityBufferLength, isValid, DEBUG_NAME); // Read offsets: final long offBufRead = (long) numOffsets * Integer.BYTES; @@ -93,8 +83,8 @@ public WritableObjectChunk readChunk( } // it doesn't make sense to have a struct validity buffer for a map - if (structValiadityBufferLength > 0) { - is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, structValiadityBufferLength)); + if (structValidityBufferLength > 0) { + is.skipBytes(LongSizedDataStructure.intSize(DEBUG_NAME, structValidityBufferLength)); } try (final WritableChunk keysPrim = @@ -123,7 +113,7 @@ public WritableObjectChunk readChunk( chunk.set(outOffset + ii, null); } else { final ImmutableMap.Builder mapBuilder = ImmutableMap.builder(); - for (jj = offsets.get(ii); jj < offsets.get(ii + 1); ++jj) { + for (int jj = offsets.get(ii); jj < offsets.get(ii + 1); ++jj) { mapBuilder.put(keys.get(jj), values.get(jj)); } // noinspection unchecked @@ -135,4 +125,5 @@ public WritableObjectChunk readChunk( return chunk; } + } diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkWriter.java index eb7b68cabd1..14edd16a31d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkWriter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/MapChunkWriter.java @@ -12,12 +12,14 @@ import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderSequential; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.impl.util.unboxer.ChunkUnboxer; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.util.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -40,7 +42,7 @@ public MapChunkWriter( final ChunkType keyWriterChunkType, final ChunkType valueWriterChunkType, final boolean fieldNullable) { - super(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, 0, false, fieldNullable); + super(null, ObjectChunk::getEmptyChunk, 0, false, fieldNullable); this.keyWriter = keyWriter; this.valueWriter = valueWriter; this.keyWriterChunkType = keyWriterChunkType; @@ -54,10 +56,33 @@ public Context makeContext( return new Context(chunk, rowOffset); } - public final class Context extends ChunkWriter.Context> { + @Override + protected int computeNullCount( + @NotNull final BaseChunkWriter.Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asObjectChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final BaseChunkWriter.Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asObjectChunk().isNull((int) row)); + }); + } + + public final class Context extends ChunkWriter.Context { private final WritableIntChunk offsets; - private final ChunkWriter.Context> keyContext; - private final ChunkWriter.Context> valueContext; + private final ChunkWriter.Context keyContext; + private final ChunkWriter.Context valueContext; public Context( @NotNull final ObjectChunk chunk, @@ -127,9 +152,10 @@ protected void onReferenceCountAtZero() { @Override public DrainableColumn getInputStream( - @NotNull final ChunkWriter.Context> context, + @NotNull final ChunkWriter.Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { + // noinspection unchecked return new MapChunkInputStream((Context) context, subset, options); } @@ -246,11 +272,11 @@ protected int getRawSize() throws IOException { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } - read = true; + hasBeenRead = true; long bytesWritten = 0; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity array with LSB indexing 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 index c1f56c5512c..f43f74b1266 100644 --- 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 @@ -5,6 +5,7 @@ import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.extensions.barrage.BarrageOptions; import org.jetbrains.annotations.NotNull; @@ -13,41 +14,70 @@ import java.io.IOException; import java.io.OutputStream; -public class NullChunkWriter> extends BaseChunkWriter { - public static final NullChunkWriter> INSTANCE = new NullChunkWriter<>(); +/** + * A {@link ChunkWriter} implementation that writes an Apache Arrow Null Column; which only writes a field node. + */ +public class NullChunkWriter extends BaseChunkWriter> { + private static final String DEBUG_NAME = "NullChunkWriter"; + + public static final NullChunkWriter INSTANCE = new NullChunkWriter(); public NullChunkWriter() { - super((chunk, idx) -> true, () -> null, 0, true, true); + super(null, () -> null, 0, true, true); } @Override public DrainableColumn getInputStream( - @NotNull final Context chunk, + @NotNull final Context chunk, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { - return new NullDrainableColumn(); + return new NullDrainableColumn(subset == null ? chunk.size() : subset.intSize(DEBUG_NAME)); + } + + @Override + protected int computeNullCount(@NotNull final Context context, @NotNull final RowSequence subset) { + return subset.intSize("NullChunkWriter"); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + // nothing to do; this is a null column } public static class NullDrainableColumn extends DrainableColumn { + private final int size; + + public NullDrainableColumn(int size) { + this.size = size; + } @Override - public void visitFieldNodes(FieldNodeListener listener) {} + public void visitFieldNodes(FieldNodeListener listener) { + listener.noteLogicalFieldNode(size, size); + } @Override - public void visitBuffers(BufferListener listener) {} + public void visitBuffers(BufferListener listener) { + // there are no buffers for null columns + } @Override public int nullCount() { - return 0; + return size; } @Override public int drainTo(final OutputStream outputStream) throws IOException { + // we only write the field node, so there is nothing to drain return 0; } @Override public int available() throws IOException { + // we only write the field node, so there is no data available return 0; } } 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 6016025ecde..928e1ac445a 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 @@ -115,19 +115,7 @@ public WritableShortChunk readChunk( final int numValidityLongs = options.useDeephavenNulls() ? 0 : (nodeInfo.numElements + 63) / 64; try (final WritableLongChunk isValid = WritableLongChunk.makeWritableChunk(numValidityLongs)) { - 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 + readValidityBuffer(is, numValidityLongs, validityBuffer, isValid, DEBUG_NAME); final long payloadRead = (long) nodeInfo.numElements * Short.BYTES; Assert.geq(payloadBuffer, "payloadBuffer", payloadRead, "payloadRead"); 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 index de6f725b71f..f15200ef09b 100644 --- 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 @@ -8,13 +8,18 @@ package io.deephaven.extensions.barrage.chunk; import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.ObjectChunk; +import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import com.google.common.io.LittleEndianDataOutputStream; import io.deephaven.UncheckedDeephavenException; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import io.deephaven.chunk.ShortChunk; +import io.deephaven.util.mutable.MutableInt; +import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -25,42 +30,64 @@ public class ShortChunkWriter> extends BaseChunkWriter { private static final String DEBUG_NAME = "ShortChunkWriter"; private static final ShortChunkWriter> NULLABLE_IDENTITY_INSTANCE = new ShortChunkWriter<>( - ShortChunk::isNull, ShortChunk::getEmptyChunk, ShortChunk::get, true); + null, ShortChunk::getEmptyChunk, true); private static final ShortChunkWriter> NON_NULLABLE_IDENTITY_INSTANCE = new ShortChunkWriter<>( - ShortChunk::isNull, ShortChunk::getEmptyChunk, ShortChunk::get, false); - + null, ShortChunk::getEmptyChunk, false); public static ShortChunkWriter> getIdentity(boolean isNullable) { return isNullable ? NULLABLE_IDENTITY_INSTANCE : NON_NULLABLE_IDENTITY_INSTANCE; } - @FunctionalInterface - public interface ToShortTransformFunction> { - short get(SourceChunkType sourceValues, int offset); + public static WritableShortChunk chunkUnboxer( + @NotNull final ObjectChunk sourceValues) { + final WritableShortChunk output = WritableShortChunk.makeWritableChunk(sourceValues.size()); + for (int ii = 0; ii < sourceValues.size(); ++ii) { + output.set(ii, TypeUtils.unbox(sourceValues.get(ii))); + } + return output; } - private final ToShortTransformFunction transform; - public ShortChunkWriter( - @NotNull final IsRowNullProvider isRowNullProvider, + @Nullable final ChunkTransformer transformer, @NotNull final Supplier emptyChunkSupplier, - @Nullable final ToShortTransformFunction transform, final boolean fieldNullable) { - super(isRowNullProvider, emptyChunkSupplier, Short.BYTES, true, fieldNullable); - this.transform = transform; + super(transformer, emptyChunkSupplier, Short.BYTES, true, fieldNullable); } @Override public DrainableColumn getInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { return new ShortChunkInputStream(context, subset, options); } - private class ShortChunkInputStream extends BaseChunkInputStream> { + @Override + protected int computeNullCount( + @NotNull final Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asShortChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asShortChunk().isNull((int) row)); + }); + } + + private class ShortChunkInputStream extends BaseChunkInputStream { private ShortChunkInputStream( - @NotNull final Context context, + @NotNull final Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) { super(context, subset, options); @@ -81,12 +108,12 @@ public void visitBuffers(final BufferListener listener) { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } long bytesWritten = 0; - read = true; + hasBeenRead = true; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // write the validity buffer @@ -95,7 +122,7 @@ public int drainTo(final OutputStream outputStream) throws IOException { // write the payload buffer subset.forAllRowKeys(row -> { try { - dos.writeShort(transform.get(context.getChunk(), (int) row)); + dos.writeShort(context.getChunk().asShortChunk().get((int) row)); } catch (final IOException e) { throw new UncheckedDeephavenException( "Unexpected exception while draining data to OutputStream: ", e); diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderReader.java index 7661b9e9cb3..6c238f65b72 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderReader.java @@ -5,6 +5,7 @@ import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.datastructures.LongSizedDataStructure; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -14,6 +15,21 @@ import java.util.Iterator; import java.util.PrimitiveIterator; +/** + * The {@code SingleElementListHeaderReader} is a specialized {@link BaseChunkReader} used to handle singleton + * list-wrapped columns in Apache Arrow record batches. This implementation ensures compatibility with Apache Arrow's + * requirement that top-level column vectors must have the same number of rows, even when some columns in a record batch + * contain varying numbers of modified rows. + *

+ * This reader works by skipping the validity and offset buffers for the singleton list and delegating the reading of + * the underlying data to a {@link ChunkReader} for the wrapped component type. This approach ensures that Arrow + * payloads remain compatible with official Arrow implementations while supporting Deephaven's semantics for record + * batches with varying column modifications. + *

+ * This is used only when {@link BarrageOptions#columnsAsList()} is enabled. + * + * @param The type of chunk being read, extending {@link WritableChunk} with {@link Values}. + */ public class SingleElementListHeaderReader> extends BaseChunkReader { private static final String DEBUG_NAME = "SingleElementListHeaderReader"; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java index 4387644e361..c235a69f4b9 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/SingleElementListHeaderWriter.java @@ -12,7 +12,14 @@ import java.io.OutputStream; /** - * This helper class is used to generate only the header of an arrow list that contains a single element. + * The {@code SingleElementListHeaderWriter} is a specialized {@link DrainableColumn} implementation that writes the + * header for singleton list-wrapped columns in Apache Arrow record batches. + *

+ * This writer ensures compatibility with Apache Arrow's format by providing the necessary metadata and offsets for a + * single-element list, while omitting unnecessary buffers such as validity buffers. It is designed to write the header + * information for a column where all rows are represented as a singleton list, with no null values. + * + * @see SingleElementListHeaderReader */ public class SingleElementListHeaderWriter extends DrainableColumn { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/UnionChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/UnionChunkWriter.java index 41f1dd74f23..2d3cb86e35f 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/UnionChunkWriter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/UnionChunkWriter.java @@ -12,11 +12,15 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Values; +import io.deephaven.chunk.sized.SizedChunk; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.util.unboxer.ChunkUnboxer; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.BooleanUtils; +import io.deephaven.util.QueryConstants; import io.deephaven.util.datastructures.LongSizedDataStructure; +import io.deephaven.util.mutable.MutableInt; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -38,7 +42,7 @@ public UnionChunkWriter( final List> classMatchers, final List>> writers, final List writerChunkTypes) { - super(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, 0, false, false); + super(null, ObjectChunk::getEmptyChunk, 0, false, false); this.mode = mode; this.classMatchers = classMatchers; this.writers = writers; @@ -54,7 +58,30 @@ public Context makeContext( return new Context(chunk, rowOffset); } - public final class Context extends ChunkWriter.Context> { + @Override + protected int computeNullCount( + @NotNull final ChunkWriter.Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asObjectChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal( + @NotNull final ChunkWriter.Context context, + @NotNull final RowSequence subset, + @NotNull final SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asCharChunk().isNull((int) row)); + }); + } + + public final class Context extends ChunkWriter.Context { public Context( @NotNull final ObjectChunk chunk, final long rowOffset) { @@ -64,9 +91,10 @@ public Context( @Override public DrainableColumn getInputStream( - @NotNull final ChunkWriter.Context> context, + @NotNull final ChunkWriter.Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { + // noinspection unchecked return new UnionChunkInputStream((Context) context, subset, options); } @@ -83,7 +111,7 @@ private UnionChunkInputStream( @NotNull final BarrageOptions options) throws IOException { super(context, mySubset, options); final int numColumns = classMatchers.size(); - final ObjectChunk chunk = context.getChunk(); + final ObjectChunk chunk = context.getChunk().asObjectChunk(); if (mode == UnionChunkReader.Mode.Sparse) { columnOffset = null; } else { @@ -95,15 +123,16 @@ private UnionChunkInputStream( // noinspection resource columnOfInterest = WritableByteChunk.makeWritableChunk(chunk.size()); // noinspection unchecked - final WritableObjectChunk[] innerChunks = new WritableObjectChunk[numColumns]; + final SizedChunk[] innerChunks = new SizedChunk[numColumns]; for (int ii = 0; ii < numColumns; ++ii) { // noinspection resource - innerChunks[ii] = WritableObjectChunk.makeWritableChunk(chunk.size()); + innerChunks[ii] = new SizedChunk<>(ChunkType.Object); if (mode == UnionChunkReader.Mode.Sparse) { - innerChunks[ii].fillWithNullValue(0, chunk.size()); + innerChunks[ii].ensureCapacity(chunk.size()); + innerChunks[ii].get().fillWithNullValue(0, chunk.size()); } else { - innerChunks[ii].setSize(0); + innerChunks[ii].ensureCapacity(0); } } for (int ii = 0; ii < chunk.size(); ++ii) { @@ -113,11 +142,16 @@ private UnionChunkInputStream( if (value.getClass().isAssignableFrom(classMatchers.get(jj))) { if (mode == UnionChunkReader.Mode.Sparse) { columnOfInterest.set(ii, (byte) jj); - innerChunks[jj].set(ii, value); + innerChunks[jj].get().asWritableObjectChunk().set(ii, value); } else { columnOfInterest.set(ii, (byte) jj); - columnOffset.set(ii, innerChunks[jj].size()); - innerChunks[jj].add(value); + int size = innerChunks[jj].get().size(); + columnOffset.set(ii, size); + if (innerChunks[jj].get().capacity() <= size) { + int newSize = Math.max(16, size * 2); + innerChunks[jj].ensureCapacityPreserve(newSize); + } + innerChunks[jj].get().asWritableObjectChunk().add(value); } break; } @@ -134,7 +168,7 @@ private UnionChunkInputStream( for (int ii = 0; ii < numColumns; ++ii) { final ChunkType chunkType = writerChunkTypes.get(ii); final ChunkWriter> writer = writers.get(ii); - final WritableObjectChunk innerChunk = innerChunks[ii]; + final WritableObjectChunk innerChunk = innerChunks[ii].get().asWritableObjectChunk(); if (classMatchers.get(ii) == Boolean.class) { // do a quick conversion to byte since the boolean unboxer expects bytes @@ -149,7 +183,7 @@ private UnionChunkInputStream( : ChunkUnboxer.getUnboxer(chunkType, innerChunk.size()); // noinspection unchecked - try (ChunkWriter.Context> innerContext = writer.makeContext(kernel != null + try (ChunkWriter.Context innerContext = writer.makeContext(kernel != null ? (Chunk) kernel.unbox(innerChunk) : innerChunk, 0)) { if (kernel != null) { @@ -211,11 +245,11 @@ protected int getRawSize() throws IOException { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } - read = true; + hasBeenRead = true; long bytesWritten = 0; final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); // must write out the column of interest diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java index d509cbbac51..ce14c54cd06 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkWriter.java @@ -9,6 +9,7 @@ import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; import io.deephaven.chunk.util.pools.ChunkPoolConstants; +import io.deephaven.engine.rowset.RowSequence; import io.deephaven.extensions.barrage.BarrageOptions; import io.deephaven.util.SafeCloseable; import io.deephaven.util.datastructures.LongSizedDataStructure; @@ -35,15 +36,16 @@ public interface Appender { public VarBinaryChunkWriter( final boolean fieldNullable, final Appender appendItem) { - super(ObjectChunk::isNull, ObjectChunk::getEmptyChunk, 0, false, fieldNullable); + super(null, ObjectChunk::getEmptyChunk, 0, false, fieldNullable); this.appendItem = appendItem; } @Override public DrainableColumn getInputStream( - @NotNull final ChunkWriter.Context> context, + @NotNull final ChunkWriter.Context context, @Nullable final RowSet subset, @NotNull final BarrageOptions options) throws IOException { + // noinspection unchecked return new ObjectChunkInputStream((Context) context, subset, options); } @@ -54,7 +56,28 @@ public Context makeContext( return new Context(chunk, rowOffset); } - public final class Context extends ChunkWriter.Context> { + @Override + protected int computeNullCount( + @NotNull final ChunkWriter.Context context, + @NotNull final RowSequence subset) { + final MutableInt nullCount = new MutableInt(0); + subset.forAllRowKeys(row -> { + if (context.getChunk().asObjectChunk().isNull((int) row)) { + nullCount.increment(); + } + }); + return nullCount.get(); + } + + @Override + protected void writeValidityBufferInternal(ChunkWriter.@NotNull Context context, @NotNull RowSequence subset, + @NotNull SerContext serContext) { + subset.forAllRowKeys(row -> { + serContext.setNextIsNull(context.getChunk().asObjectChunk().isNull((int) row)); + }); + } + + public final class Context extends ChunkWriter.Context { private final ByteStorage byteStorage; public Context( @@ -156,11 +179,11 @@ protected int getRawSize() { @Override public int drainTo(final OutputStream outputStream) throws IOException { - if (read || subset.isEmpty()) { + if (hasBeenRead || subset.isEmpty()) { return 0; } - read = true; + hasBeenRead = true; final MutableLong bytesWritten = new MutableLong(); final LittleEndianDataOutputStream dos = new LittleEndianDataOutputStream(outputStream); 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 4ad85fec0aa..d9ec9ce5181 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 @@ -6,9 +6,28 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.extensions.barrage.chunk.ExpansionKernel; +/** + * The {@code ArrayExpansionKernel} interface provides a mechanism for expanding chunks containing arrays into a pair of + * {@code LongChunk} and {@code Chunk}, enabling efficient handling of array-typed columnar data. This interface is + * part of the Deephaven Barrage extensions for processing structured data in Flight/Barrage streams. + *

+ * An {@code ArrayExpansionKernel} is specialized for handling array-like data, where each element in the source chunk + * may itself be an array. The kernel performs the transformation to a flattened format, suitable for further processing + * or serialization. + * + * @param The type of elements within the array being expanded. + */ public interface ArrayExpansionKernel extends ExpansionKernel { /** - * @return a kernel that expands a {@code Chunk} to pair of {@code LongChunk, Chunk} + * Creates an {@code ArrayExpansionKernel} for the specified {@link ChunkType} and component type. + *

+ * The implementation is chosen based on the provided {@code chunkType} and {@code componentType}, with specialized + * kernels for primitive types and boxed types, including {@code boolean} handling for packed bit representations. + * + * @param chunkType The {@link ChunkType} representing the type of data in the chunk. + * @param componentType The class of the component type within the array. + * @param The type of elements within the array being expanded. + * @return An {@code ArrayExpansionKernel} capable of expanding chunks of the specified type. */ @SuppressWarnings("unchecked") static ArrayExpansionKernel makeExpansionKernel(final ChunkType chunkType, final Class componentType) { 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 d6919230207..fef9c1e7ac3 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 @@ -103,7 +103,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 7c8b8645874..276be31ab72 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 @@ -103,7 +103,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 b6048257d60..6321721de6c 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 @@ -100,7 +100,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 f09d69e80b5..b47aa571eef 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 @@ -96,7 +96,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -112,6 +112,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 0dd62e4e721..1bd6bc71263 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 @@ -100,7 +100,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 4e5963ec132..b73785ff52e 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 @@ -100,7 +100,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 c0909ad37f7..6fa92783e3c 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 @@ -100,7 +100,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 7a19b14bd9d..28014227c4b 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 @@ -100,7 +100,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 fbe0c59c1c2..16b27dc3760 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 @@ -99,7 +99,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -115,6 +115,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 c01f8518ddc..ca4be7d8778 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 @@ -100,7 +100,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -116,6 +116,7 @@ public WritableObjectChunk contract( return chunk; } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 c238460be55..294b29f1e99 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 @@ -78,7 +78,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -110,7 +110,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -123,6 +123,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 ad1901a9386..302cee9ca51 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 @@ -74,7 +74,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -106,7 +106,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -119,6 +119,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 226e981d88a..80089b419e9 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 @@ -79,7 +79,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -111,7 +111,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -124,6 +124,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 6ac0fc5da8b..e684cb49b5c 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 @@ -78,7 +78,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -110,7 +110,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -123,6 +123,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 248e40857b6..1618dde2295 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 @@ -79,7 +79,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -111,7 +111,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -124,6 +124,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 a28894dc059..90dc0233eb6 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 @@ -79,7 +79,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -111,7 +111,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -124,6 +124,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 f18de255dca..065f02398c4 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 @@ -107,7 +107,7 @@ public WritableChunk expand( @Override public WritableObjectChunk, A> contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -120,6 +120,7 @@ public WritableObjectChunk, A> contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 500161bbc79..5c6c897711e 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 @@ -78,7 +78,7 @@ public WritableChunk expand( Stream stream = iter.stream(); if (fixedSizeLength > 0) { // limit length to fixedSizeLength - stream = iter.stream().limit(fixedSizeLength); + stream = stream.limit(fixedSizeLength); } else if (fixedSizeLength < 0) { final long numToSkip = Math.max(0, row.size() + fixedSizeLength); if (numToSkip > 0) { @@ -110,7 +110,7 @@ public WritableChunk expand( @Override public WritableObjectChunk contract( @NotNull final Chunk source, - final int sizePerElement, + int sizePerElement, @Nullable final IntChunk offsets, @Nullable final IntChunk lengths, @Nullable final WritableChunk outChunk, @@ -123,6 +123,7 @@ public WritableObjectChunk contract( return WritableObjectChunk.makeWritableChunk(totalRows); } + sizePerElement = Math.abs(sizePerElement); final int itemsInBatch = offsets == null ? source.size() / sizePerElement : (offsets.size() - (lengths == null ? 1 : 0)); 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 0424f9b5a98..170d85bd4b3 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 @@ -15,6 +15,15 @@ import io.deephaven.vector.ShortVector; import io.deephaven.vector.Vector; +/** + * The {@code VectorExpansionKernel} interface provides a mechanism for expanding chunks containing {@link Vector} + * elements into a pair of {@code LongChunk} and {@code Chunk}, enabling efficient handling of vector-typed columnar + * data. This interface is part of the Deephaven Barrage extensions for processing structured data in Flight/Barrage + * streams. + * + *

+ * A {@code VectorExpansionKernel} + */ public interface VectorExpansionKernel> extends ExpansionKernel { static Class getComponentType(final Class type, final Class componentType) { 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 3381bfff648..f17110fe7ca 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 @@ -134,7 +134,7 @@ public class BarrageUtil { /** * The Apache Arrow metadata prefix for Deephaven attributes. */ - private static final String ATTR_DH_PREFIX = "deephaven:"; + public static final String ATTR_DH_PREFIX = "deephaven:"; /** * The deephaven metadata tag to indicate an attribute. @@ -149,12 +149,12 @@ public class BarrageUtil { /** * The deephaven metadata tag to indicate the deephaven column type. */ - private static final String ATTR_TYPE_TAG = "type"; + public static final String ATTR_TYPE_TAG = "type"; /** * The deephaven metadata tag to indicate the deephaven column component type. */ - private static final String ATTR_COMPONENT_TYPE_TAG = "componentType"; + public static final String ATTR_COMPONENT_TYPE_TAG = "componentType"; private static final boolean ENFORCE_FLATBUFFER_VERSION_CHECK = Configuration.getInstance().getBooleanWithDefault("barrage.version.check", true); @@ -970,7 +970,7 @@ public static void createAndSendStaticSnapshot( // noinspection unchecked final ChunkWriter>[] chunkWriters = table.getDefinition().getColumns().stream() .map(cd -> DefaultChunkWriterFactory.INSTANCE.newWriter(BarrageTypeInfo.make( - ReinterpretUtils.maybeConvertToPrimitiveDataType(cd.getDataType()), + cd.getDataType(), cd.getComponentType(), fieldFor != null ? fieldFor.get(cd.getName()) : flatbufFieldFor(cd, Map.of())))) .toArray(ChunkWriter[]::new); @@ -1035,7 +1035,7 @@ public static void createAndSendStaticSnapshot( final long targetCycleDurationMillis; final UpdateGraph updateGraph = table.getUpdateGraph(); if (updateGraph == null || updateGraph instanceof PoisonedUpdateGraph) { - targetCycleDurationMillis = PeriodicUpdateGraph.DEFAULT_TARGET_CYCLE_DURATION_MILLIS; + targetCycleDurationMillis = PeriodicUpdateGraph.getDefaultTargetCycleDurationMillis(); } else { targetCycleDurationMillis = updateGraph.cast() .getTargetCycleDurationMillis(); @@ -1082,7 +1082,7 @@ public static void createAndSendSnapshot( // noinspection unchecked final ChunkWriter>[] chunkWriters = table.getDefinition().getColumns().stream() .map(cd -> DefaultChunkWriterFactory.INSTANCE.newWriter(BarrageTypeInfo.make( - ReinterpretUtils.maybeConvertToPrimitiveDataType(cd.getDataType()), + cd.getDataType(), cd.getComponentType(), flatbufFieldFor(cd, Map.of())))) .toArray(ChunkWriter[]::new); 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 f1f20a125b1..f5d3d9c0e3f 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 @@ -12,5 +12,6 @@ + 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 f2ca898e18a..078679c5d21 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 @@ -11,6 +11,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.testutil.testcase.RefreshingTableTestCase; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderSequential; @@ -226,21 +227,23 @@ public void testLongChunkSerialization() throws IOException { for (int i = 0; i < chunk.size(); ++i) { chunk.set(i, i % 7 == 0 ? QueryConstants.NULL_LONG : random.nextLong()); } - }, (utO, utC, subset, offset) -> { - final WritableLongChunk original = utO.asWritableLongChunk(); - final WritableLongChunk computed = utC.asWritableLongChunk(); - if (subset == null) { - for (int i = 0; i < original.size(); ++i) { - Assert.equals(original.get(i), "original.get(i)", - computed.get(offset + i), "computed.get(i)"); - } - } else { - final MutableInt off = new MutableInt(); - subset.forAllRowKeys(key -> Assert.equals(original.get((int) key), "original.get(key)", - computed.get(offset + off.getAndIncrement()), - "computed.get(offset + off.getAndIncrement())")); - } - }); + }, BarrageColumnRoundTripTest::longIdentityValidator); + } + } + + private static void longIdentityValidator(WritableChunk utO, WritableChunk utC, RowSequence subset, int offset) { + final WritableLongChunk original = utO.asWritableLongChunk(); + final WritableLongChunk computed = utC.asWritableLongChunk(); + if (subset == null) { + for (int i = 0; i < original.size(); ++i) { + Assert.equals(original.get(i), "original.get(i)", + computed.get(offset + i), "computed.get(i)"); + } + } else { + final MutableInt off = new MutableInt(); + subset.forAllRowKeys(key -> Assert.equals(original.get((int) key), "original.get(key)", + computed.get(offset + off.getAndIncrement()), + "computed.get(offset + off.getAndIncrement())")); } } @@ -300,11 +303,11 @@ public void testInstantChunkSerialization() throws IOException { final Random random = new Random(0); for (final BarrageSubscriptionOptions opts : OPTIONS) { testRoundTripSerialization(opts, Instant.class, (utO) -> { - final WritableObjectChunk chunk = utO.asWritableObjectChunk(); + final WritableLongChunk chunk = utO.asWritableLongChunk(); for (int i = 0; i < chunk.size(); ++i) { - chunk.set(i, i % 7 == 0 ? null : Instant.ofEpochSecond(0, random.nextLong())); + chunk.set(i, i % 7 == 0 ? QueryConstants.NULL_LONG : random.nextLong()); } - }, new ObjectIdentityValidator<>()); + }, BarrageColumnRoundTripTest::longIdentityValidator); } } @@ -659,10 +662,14 @@ public void assertExpected( } private static void testRoundTripSerialization( - final BarrageSubscriptionOptions options, final Class type, - final Consumer> initData, final Validator validator) throws IOException { + final BarrageSubscriptionOptions options, + Class type, + final Consumer> initData, + final Validator validator) throws IOException { final int NUM_ROWS = 8; final ChunkType chunkType; + // noinspection unchecked + type = (Class) ReinterpretUtils.maybeConvertToPrimitiveDataType(type); if (type == Boolean.class || type == boolean.class) { chunkType = ChunkType.Byte; } else { @@ -691,7 +698,7 @@ private static void testRoundTripSerialization( final ChunkWriter> writer = DefaultChunkWriterFactory.INSTANCE .newWriter(BarrageTypeInfo.make(type, type.getComponentType(), field)); try (SafeCloseable ignored = srcData; - final ChunkWriter.Context> context = writer.makeContext(data, 0)) { + final ChunkWriter.Context context = writer.makeContext(data, 0)) { // full sub logic try (final ExposedByteArrayOutputStream baos = new ExposedByteArrayOutputStream(); final ChunkWriter.DrainableColumn column = writer.getInputStream(context, null, options)) { @@ -712,8 +719,8 @@ private static void testRoundTripSerialization( new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); 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); + Assert.eq(srcData.size(), "srcData.size()", rtData.size(), "rtData.size()"); + validator.assertExpected(srcData, rtData, null, 0); } } @@ -739,7 +746,7 @@ private static void testRoundTripSerialization( // swiss cheese subset final Random random = new Random(0); final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); - for (int i = 0; i < data.size(); ++i) { + for (int i = 0; i < srcData.size(); ++i) { if (random.nextBoolean()) { builder.appendKey(i); } @@ -760,7 +767,7 @@ private static void testRoundTripSerialization( 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); + validator.assertExpected(srcData, rtData, subset, 0); } } @@ -782,16 +789,17 @@ private static void testRoundTripSerialization( new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); try (final WritableChunk rtData = readChunk(options, readType, readType.getComponentType(), field, fieldNodes.iterator(), Arrays.stream(buffers).iterator(), dis, null, 0, - data.size() * 2)) { + srcData.size() * 2)) { // second message dis = new LittleEndianDataInputStream( new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); final WritableChunk rtData2 = readChunk(options, readType, readType.getComponentType(), - field, fieldNodes.iterator(), Arrays.stream(buffers).iterator(), dis, rtData, data.size(), - data.size() * 2); + field, fieldNodes.iterator(), Arrays.stream(buffers).iterator(), dis, rtData, + srcData.size(), + srcData.size() * 2); Assert.eq(rtData, "rtData", rtData2, "rtData2"); - validator.assertExpected(data, rtData, null, 0); - validator.assertExpected(data, rtData, null, data.size()); + validator.assertExpected(srcData, rtData, null, 0); + validator.assertExpected(srcData, rtData, null, srcData.size()); } } } diff --git a/go/pkg/client/example_table_ops_test.go b/go/pkg/client/example_table_ops_test.go index 00a55e8efb7..2c8e22d02df 100644 --- a/go/pkg/client/example_table_ops_test.go +++ b/go/pkg/client/example_table_ops_test.go @@ -34,7 +34,7 @@ func Example_tableOps() { fmt.Println(queryResult) - // Output: + // Output: // Data Before: // record: // schema: @@ -47,7 +47,7 @@ func Example_tableOps() { // col[1][Close]: [53.8 88.5 38.7 453 26.7 544.9 13.4] // col[2][Volume]: [87000 6060842 138000 138000000 19000 48300 1500] // - // New data: + // Data After: // record: // schema: // fields: 3 @@ -57,28 +57,39 @@ func Example_tableOps() { // metadata: ["deephaven:isDateFormat": "false", "deephaven:isNumberFormat": "false", "deephaven:isPartitioning": "false", "deephaven:isRowStyle": "false", "deephaven:isSortable": "true", "deephaven:isStyle": "false", "deephaven:type": "float"] // - Volume: type=int32, nullable // metadata: ["deephaven:isDateFormat": "false", "deephaven:isNumberFormat": "false", "deephaven:isPartitioning": "false", "deephaven:isRowStyle": "false", "deephaven:isSortable": "true", "deephaven:isStyle": "false", "deephaven:type": "int"] - // metadata: ["deephaven:attribute.AddOnly": "true", "deephaven:attribute.AppendOnly": "true", "deephaven:attribute_type.AddOnly": "java.lang.Boolean", "deephaven:attribute_type.AppendOnly": "java.lang.Boolean", "deephaven:unsent.attribute.BarrageSchema": ""] + // metadata: ["deephaven:attribute.AddOnly": "true", "deephaven:attribute.AppendOnly": "true", "deephaven:attribute.SortedColumns": "Close=Ascending", "deephaven:attribute_type.AddOnly": "java.lang.Boolean", "deephaven:attribute_type.AppendOnly": "java.lang.Boolean", "deephaven:attribute_type.SortedColumns": "java.lang.String", "deephaven:unsent.attribute.BarrageSchema": ""] // rows: 5 - // col[0][Ticker]: ["XRX" "IBM" "GME" "AAPL" "ZNGA"] - // col[1][Close]: [53.8 38.7 453 26.7 544.9] - // col[2][Volume]: [87000 138000 138000000 19000 48300] + // col[0][Ticker]: ["IBM" "XRX" "XYZZY" "GME" "ZNGA"] + // col[1][Close]: [38.7 53.8 88.5 453 544.9] + // col[2][Volume]: [138000 87000 6060842 138000000 48300] + // want: + // Data Before: + // record: + // schema: + // fields: 3 + // - Ticker: type=utf8, nullable + // - Close: type=float32, nullable + // - Volume: type=int32, nullable + // rows: 7 + // col[0][Ticker]: ["XRX" "XYZZY" "IBM" "GME" "AAPL" "ZNGA" "T"] + // col[1][Close]: [53.8 88.5 38.7 453 26.7 544.9 13.4] + // col[2][Volume]: [87000 6060842 138000 138000000 19000 48300 1500] // + // Data After: // record: // schema: - // fields: 4 + // fields: 3 // - Ticker: type=utf8, nullable // metadata: ["deephaven:isDateFormat": "false", "deephaven:isNumberFormat": "false", "deephaven:isPartitioning": "false", "deephaven:isRowStyle": "false", "deephaven:isSortable": "true", "deephaven:isStyle": "false", "deephaven:type": "java.lang.String"] // - Close: type=float32, nullable // metadata: ["deephaven:isDateFormat": "false", "deephaven:isNumberFormat": "false", "deephaven:isPartitioning": "false", "deephaven:isRowStyle": "false", "deephaven:isSortable": "true", "deephaven:isStyle": "false", "deephaven:type": "float"] // - Volume: type=int32, nullable // metadata: ["deephaven:isDateFormat": "false", "deephaven:isNumberFormat": "false", "deephaven:isPartitioning": "false", "deephaven:isRowStyle": "false", "deephaven:isSortable": "true", "deephaven:isStyle": "false", "deephaven:type": "int"] - // - Magnitude: type=int32, nullable - // metadata: ["deephaven:isDateFormat": "false", "deephaven:isNumberFormat": "false", "deephaven:isPartitioning": "false", "deephaven:isRowStyle": "false", "deephaven:isSortable": "true", "deephaven:isStyle": "false", "deephaven:type": "int"] + // metadata: ["deephaven:attribute.AddOnly": "true", "deephaven:attribute.AppendOnly": "true", "deephaven:attribute.SortedColumns": "Close=Ascending", "deephaven:attribute_type.AddOnly": "java.lang.Boolean", "deephaven:attribute_type.AppendOnly": "java.lang.Boolean", "deephaven:attribute_type.SortedColumns": "java.lang.String"] // rows: 5 - // col[0][Ticker]: ["XRX" "IBM" "GME" "AAPL" "ZNGA"] - // col[1][Close]: [53.8 38.7 453 26.7 544.9] - // col[2][Volume]: [87000 138000 138000000 19000 48300] - // col[3][Magnitude]: [10000 100000 100000000 10000 10000] + // col[0][Ticker]: ["IBM" "XRX" "XYZZY" "GME" "ZNGA"] + // col[1][Close]: [38.7 53.8 88.5 453 544.9] + // col[2][Volume]: [138000 87000 6060842 138000000 48300] } // This function demonstrates how to use immediate table operations. diff --git a/server/jetty/build.gradle b/server/jetty/build.gradle index 419f4475e80..eda1ed04419 100644 --- a/server/jetty/build.gradle +++ b/server/jetty/build.gradle @@ -55,6 +55,8 @@ dependencies { testImplementation libs.junit4 testImplementation libs.assertj + testImplementation project(':proto:proto-backplane-grpc-flight') + testRuntimeOnly project(':log-to-slf4j') testRuntimeOnly libs.slf4j.simple } diff --git a/server/jetty/src/test/java/io/deephaven/server/jetty/JettyBarrageChunkFactoryTest.java b/server/jetty/src/test/java/io/deephaven/server/jetty/JettyBarrageChunkFactoryTest.java new file mode 100644 index 00000000000..8e3a9f2c2df --- /dev/null +++ b/server/jetty/src/test/java/io/deephaven/server/jetty/JettyBarrageChunkFactoryTest.java @@ -0,0 +1,500 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.server.jetty; + +import dagger.Component; +import dagger.Module; +import dagger.Provides; +import dagger.multibindings.IntoSet; +import io.deephaven.UncheckedDeephavenException; +import io.deephaven.auth.AuthContext; +import io.deephaven.base.clock.Clock; +import io.deephaven.client.impl.BearerHandler; +import io.deephaven.client.impl.Session; +import io.deephaven.client.impl.SessionConfig; +import io.deephaven.client.impl.SessionImpl; +import io.deephaven.client.impl.SessionImplConfig; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.updategraph.OperationInitializer; +import io.deephaven.engine.updategraph.UpdateGraph; +import io.deephaven.engine.util.AbstractScriptSession; +import io.deephaven.engine.util.NoLanguageDeephavenSession; +import io.deephaven.engine.util.ScriptSession; +import io.deephaven.engine.util.TableTools; +import io.deephaven.extensions.barrage.util.BarrageUtil; +import io.deephaven.io.logger.LogBuffer; +import io.deephaven.io.logger.LogBufferGlobal; +import io.deephaven.plugin.Registration; +import io.deephaven.proto.flight.util.FlightExportTicketHelper; +import io.deephaven.server.arrow.ArrowModule; +import io.deephaven.server.auth.AuthorizationProvider; +import io.deephaven.server.config.ConfigServiceModule; +import io.deephaven.server.console.ConsoleModule; +import io.deephaven.server.console.ScopeTicketResolver; +import io.deephaven.server.log.LogModule; +import io.deephaven.server.plugin.PluginsModule; +import io.deephaven.server.runner.ExecutionContextUnitTestModule; +import io.deephaven.server.runner.GrpcServer; +import io.deephaven.server.runner.MainHelper; +import io.deephaven.server.session.ObfuscatingErrorTransformerModule; +import io.deephaven.server.session.SessionModule; +import io.deephaven.server.session.SessionService; +import io.deephaven.server.session.SessionServiceGrpcImpl; +import io.deephaven.server.session.SessionState; +import io.deephaven.server.session.TicketResolver; +import io.deephaven.server.table.TableModule; +import io.deephaven.server.test.FlightMessageRoundTripTest; +import io.deephaven.server.test.TestAuthModule; +import io.deephaven.server.test.TestAuthorizationProvider; +import io.deephaven.server.util.Scheduler; +import io.deephaven.util.SafeCloseable; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.ClientCall; +import io.grpc.ClientInterceptor; +import io.grpc.ManagedChannel; +import io.grpc.ManagedChannelBuilder; +import io.grpc.MethodDescriptor; +import io.grpc.ServerInterceptor; +import org.apache.arrow.flight.AsyncPutListener; +import org.apache.arrow.flight.CallHeaders; +import org.apache.arrow.flight.CallStatus; +import org.apache.arrow.flight.FlightClient; +import org.apache.arrow.flight.FlightClientMiddleware; +import org.apache.arrow.flight.FlightDescriptor; +import org.apache.arrow.flight.FlightStream; +import org.apache.arrow.flight.Location; +import org.apache.arrow.flight.Ticket; +import org.apache.arrow.flight.auth2.Auth2Constants; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.Types; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExternalResource; + +import javax.inject.Named; +import javax.inject.Singleton; +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class JettyBarrageChunkFactoryTest { + private static final String COLUMN_NAME = "test_col"; + + @Module + public interface JettyTestConfig { + @Provides + static JettyConfig providesJettyConfig() { + return JettyConfig.builder() + .port(0) + .tokenExpire(Duration.of(5, ChronoUnit.MINUTES)) + .build(); + } + } + + @Singleton + @Component(modules = { + ExecutionContextUnitTestModule.class, + FlightMessageRoundTripTest.FlightTestModule.class, + JettyServerModule.class, + JettyFlightRoundTripTest.JettyTestConfig.class, + }) + public interface JettyTestComponent extends FlightMessageRoundTripTest.TestComponent { + } + + @Module(includes = { + ArrowModule.class, + ConfigServiceModule.class, + ConsoleModule.class, + LogModule.class, + SessionModule.class, + TableModule.class, + TestAuthModule.class, + ObfuscatingErrorTransformerModule.class, + PluginsModule.class, + }) + public static class FlightTestModule { + @IntoSet + @Provides + TicketResolver ticketResolver(ScopeTicketResolver resolver) { + return resolver; + } + + @Singleton + @Provides + AbstractScriptSession provideAbstractScriptSession( + final UpdateGraph updateGraph, + final OperationInitializer operationInitializer) { + return new NoLanguageDeephavenSession( + updateGraph, operationInitializer, "non-script-session"); + } + + @Provides + ScriptSession provideScriptSession(AbstractScriptSession scriptSession) { + return scriptSession; + } + + @Provides + Scheduler provideScheduler() { + return new Scheduler.DelegatingImpl( + Executors.newSingleThreadExecutor(), + Executors.newScheduledThreadPool(1), + Clock.system()); + } + + @Provides + @Named("session.tokenExpireMs") + long provideTokenExpireMs() { + return 60_000_000; + } + + @Provides + @Named("http.port") + int provideHttpPort() { + return 0;// 'select first available' + } + + @Provides + @Named("grpc.maxInboundMessageSize") + int provideMaxInboundMessageSize() { + return 1024 * 1024; + } + + @Provides + @Nullable + ScheduledExecutorService provideExecutorService() { + return null; + } + + @Provides + AuthorizationProvider provideAuthorizationProvider(TestAuthorizationProvider provider) { + return provider; + } + + @Provides + @Singleton + TestAuthorizationProvider provideTestAuthorizationProvider() { + return new TestAuthorizationProvider(); + } + + @Provides + @Singleton + static UpdateGraph provideUpdateGraph() { + return ExecutionContext.getContext().getUpdateGraph(); + } + + @Provides + @Singleton + static OperationInitializer provideOperationInitializer() { + return ExecutionContext.getContext().getOperationInitializer(); + } + } + + public interface TestComponent { + Set interceptors(); + + SessionServiceGrpcImpl sessionGrpcService(); + + SessionService sessionService(); + + GrpcServer server(); + + TestAuthModule.BasicAuthTestImpl basicAuthHandler(); + + ExecutionContext executionContext(); + + TestAuthorizationProvider authorizationProvider(); + + Registration.Callback registration(); + } + + private LogBuffer logBuffer; + private GrpcServer server; + private int localPort; + private FlightClient flightClient; + private BufferAllocator allocator; + + protected SessionService sessionService; + + private SessionState currentSession; + private SafeCloseable executionContext; + private Location serverLocation; + private FlightMessageRoundTripTest.TestComponent component; + + private ManagedChannel clientChannel; + private ScheduledExecutorService clientScheduler; + private Session clientSession; + + private int nextTicket = 1; + + @BeforeClass + public static void setupOnce() throws IOException { + MainHelper.bootstrapProjectDirectories(); + } + + @Before + public void setup() throws IOException, InterruptedException { + logBuffer = new LogBuffer(128); + LogBufferGlobal.setInstance(logBuffer); + + component = DaggerJettyBarrageChunkFactoryTest_JettyTestComponent.create(); + // open execution context immediately so it can be used when resolving `scriptSession` + executionContext = component.executionContext().open(); + + server = component.server(); + server.start(); + localPort = server.getPort(); + + sessionService = component.sessionService(); + + serverLocation = Location.forGrpcInsecure("localhost", localPort); + currentSession = sessionService.newSession(new AuthContext.SuperUser()); + allocator = new RootAllocator(); + flightClient = FlightClient.builder().location(serverLocation) + .allocator(allocator).intercept(info -> new FlightClientMiddleware() { + @Override + public void onBeforeSendingHeaders(CallHeaders outgoingHeaders) { + String token = currentSession.getExpiration().token.toString(); + outgoingHeaders.insert("Authorization", Auth2Constants.BEARER_PREFIX + token); + } + + @Override + public void onHeadersReceived(CallHeaders incomingHeaders) {} + + @Override + public void onCallCompleted(CallStatus status) {} + }).build(); + + clientChannel = ManagedChannelBuilder.forTarget("localhost:" + localPort) + .usePlaintext() + .intercept(new TestAuthClientInterceptor(currentSession.getExpiration().token.toString())) + .build(); + + clientScheduler = Executors.newSingleThreadScheduledExecutor(); + + clientSession = SessionImpl + .create(SessionImplConfig.from(SessionConfig.builder().build(), clientChannel, clientScheduler)); + } + + private static final class TestAuthClientInterceptor implements ClientInterceptor { + final BearerHandler callCredentials = new BearerHandler(); + + public TestAuthClientInterceptor(String bearerToken) { + callCredentials.setBearerToken(bearerToken); + } + + @Override + public ClientCall interceptCall(MethodDescriptor method, + CallOptions callOptions, Channel next) { + return next.newCall(method, callOptions.withCallCredentials(callCredentials)); + } + } + + @After + public void teardown() { + clientSession.close(); + clientScheduler.shutdownNow(); + clientChannel.shutdownNow(); + + sessionService.closeAllSessions(); + executionContext.close(); + + closeClient(); + server.stopWithTimeout(1, TimeUnit.MINUTES); + + try { + server.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } finally { + server = null; + } + + LogBufferGlobal.clear(logBuffer); + } + + private void closeClient() { + try { + flightClient.close(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + @Rule + public final ExternalResource livenessRule = new ExternalResource() { + SafeCloseable scope; + + @Override + protected void before() { + scope = LivenessScopeStack.open(); + } + + @Override + protected void after() { + if (scope != null) { + scope.close(); + scope = null; + } + } + }; + + private void fullyReadStream(Ticket ticket, boolean expectError) { + try (final FlightStream stream = flightClient.getStream(ticket)) { + // noinspection StatementWithEmptyBody + while (stream.next()); + if (expectError) { + fail("expected error"); + } + } catch (Exception ignored) { + } + } + + private Schema createSchema(final ArrowType arrowType, final Class dhType) { + return createSchema(arrowType, dhType, null); + } + + private Schema createSchema(final ArrowType arrowType, final Class dhType, final Class dhComponentType) { + final Map attrs = new HashMap<>(); + attrs.put(BarrageUtil.ATTR_DH_PREFIX + BarrageUtil.ATTR_TYPE_TAG, dhType.getCanonicalName()); + if (dhComponentType != null) { + attrs.put(BarrageUtil.ATTR_DH_PREFIX + BarrageUtil.ATTR_COMPONENT_TYPE_TAG, + dhComponentType.getCanonicalName()); + } + final FieldType fieldType = new FieldType(true, arrowType, null, attrs); + return new Schema(Collections.singletonList( + new Field(COLUMN_NAME, fieldType, null))); + } + + @Test + public void testInt8() throws Exception { + final int numRows = 16; + final Consumer setupData = source -> { + IntVector vector = (IntVector) source.getFieldVectors().get(0); + for (int ii = 0; ii < numRows; ++ii) { + if (ii % 2 == 0) { + vector.setNull(ii); + } else { + vector.set(ii, (byte) (ii - 8)); + } + } + source.setRowCount(numRows); + }; + final BiConsumer validator = (source, dest) -> { + IntVector sVector = (IntVector) source.getVector(0); + IntVector dVector = (IntVector) dest.getVector(0); + for (int ii = 0; ii < numRows; ii++) { + if (ii % 2 == 0) { + assertTrue(dVector.isNull(ii)); + } else { + assertEquals(sVector.get(ii), dVector.get(ii)); + } + } + }; + final Consumer> runForDhType = dhType -> { + Schema schema = createSchema(Types.MinorType.INT.getType(), dhType); + testRoundTrip(dhType, null, schema, setupData, validator); + }; + + runForDhType.accept(byte.class); +// runForDhType.accept(char.class); + runForDhType.accept(short.class); + runForDhType.accept(int.class); + runForDhType.accept(long.class); + runForDhType.accept(float.class); + runForDhType.accept(double.class); + runForDhType.accept(BigInteger.class); + runForDhType.accept(BigDecimal.class); + } + + private void testRoundTrip( + @NotNull final Class dhType, + @Nullable final Class componentType, + @NotNull final Schema schema, + @NotNull final Consumer setupData, + @NotNull final BiConsumer validator) { + try (VectorSchemaRoot source = VectorSchemaRoot.create(schema, allocator)) { + source.allocateNew(); + setupData.accept(source); + + int flightDescriptorTicketValue = nextTicket++; + FlightDescriptor descriptor = FlightDescriptor.path("export", flightDescriptorTicketValue + ""); + FlightClient.ClientStreamListener putStream = flightClient.startPut(descriptor, source, new AsyncPutListener()); + putStream.putNext(); + putStream.completed(); + + // get the table that was uploaded, and confirm it matches what we originally sent + CompletableFuture tableFuture = new CompletableFuture<>(); + SessionState.ExportObject
tableExport = currentSession.getExport(flightDescriptorTicketValue); + currentSession.nonExport() + .onErrorHandler(exception -> tableFuture.cancel(true)) + .require(tableExport) + .submit(() -> tableFuture.complete(tableExport.get())); + + // block until we're done, so we can get the table and see what is inside + putStream.getResult(); + Table uploadedTable = tableFuture.get(); + assertEquals(source.getRowCount(), uploadedTable.size()); + assertEquals(1, uploadedTable.getColumnSourceMap().size()); + ColumnSource columnSource = uploadedTable.getColumnSource(COLUMN_NAME); + assertNotNull(columnSource); + assertEquals(columnSource.getType(), dhType); + assertEquals(columnSource.getComponentType(), componentType); + + try (FlightStream stream = flightClient.getStream(flightTicketFor(flightDescriptorTicketValue))) { + VectorSchemaRoot dest = stream.getRoot(); + + int numPayloads = 0; + while (stream.next()) { + assertEquals(source.getRowCount(), dest.getRowCount()); + validator.accept(source, dest); + ++numPayloads; + } + + assertEquals(1, numPayloads); + } + } catch (Exception err) { + throw new UncheckedDeephavenException("round trip test failure", err); + } + } + + private static Ticket flightTicketFor(int flightDescriptorTicketValue) { + return new Ticket(FlightExportTicketHelper.exportIdToFlightTicket(flightDescriptorTicketValue).getTicket() + .toByteArray()); + } +} diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_012dc568a40e08aeb849b71227532f8ebe42accea1f4bbe4a7e3b8c7f433ff9cv64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_012dc568a40e08aeb849b71227532f8ebe42accea1f4bbe4a7e3b8c7f433ff9cv64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..35b25183f5b Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_012dc568a40e08aeb849b71227532f8ebe42accea1f4bbe4a7e3b8c7f433ff9cv64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_012dc568a40e08aeb849b71227532f8ebe42accea1f4bbe4a7e3b8c7f433ff9cv64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_012dc568a40e08aeb849b71227532f8ebe42accea1f4bbe4a7e3b8c7f433ff9cv64_0/Formula.class new file mode 100644 index 00000000000..4b94afb1edc Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_012dc568a40e08aeb849b71227532f8ebe42accea1f4bbe4a7e3b8c7f433ff9cv64_0/Formula.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_4e4c6857a5b4178aa7be3875b46d075b3a7c11b827374e96f98cea9d064428fcv64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_4e4c6857a5b4178aa7be3875b46d075b3a7c11b827374e96f98cea9d064428fcv64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..65721fa6542 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_4e4c6857a5b4178aa7be3875b46d075b3a7c11b827374e96f98cea9d064428fcv64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_4e4c6857a5b4178aa7be3875b46d075b3a7c11b827374e96f98cea9d064428fcv64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_4e4c6857a5b4178aa7be3875b46d075b3a7c11b827374e96f98cea9d064428fcv64_0/Formula.class new file mode 100644 index 00000000000..8fa7786004e Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_4e4c6857a5b4178aa7be3875b46d075b3a7c11b827374e96f98cea9d064428fcv64_0/Formula.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_673008ef261faf1b24552c7eebcc2ab0541a8efe127fe785886df2cb8b73b4b0v64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_673008ef261faf1b24552c7eebcc2ab0541a8efe127fe785886df2cb8b73b4b0v64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..b700916b7aa Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_673008ef261faf1b24552c7eebcc2ab0541a8efe127fe785886df2cb8b73b4b0v64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_673008ef261faf1b24552c7eebcc2ab0541a8efe127fe785886df2cb8b73b4b0v64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_673008ef261faf1b24552c7eebcc2ab0541a8efe127fe785886df2cb8b73b4b0v64_0/Formula.class new file mode 100644 index 00000000000..32e7c9ab7cb Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_673008ef261faf1b24552c7eebcc2ab0541a8efe127fe785886df2cb8b73b4b0v64_0/Formula.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c88a20a36eff96f6e498144f56f8a303d3f649602ac336ea7143a3004a74850bv64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c88a20a36eff96f6e498144f56f8a303d3f649602ac336ea7143a3004a74850bv64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..e16bbb7397d Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c88a20a36eff96f6e498144f56f8a303d3f649602ac336ea7143a3004a74850bv64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c88a20a36eff96f6e498144f56f8a303d3f649602ac336ea7143a3004a74850bv64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c88a20a36eff96f6e498144f56f8a303d3f649602ac336ea7143a3004a74850bv64_0/Formula.class new file mode 100644 index 00000000000..9456786d522 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c88a20a36eff96f6e498144f56f8a303d3f649602ac336ea7143a3004a74850bv64_0/Formula.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c8f9bf468d6a56caeae53546d948fab7d54706d9c674dc5e943d94d3aa7390ffv64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c8f9bf468d6a56caeae53546d948fab7d54706d9c674dc5e943d94d3aa7390ffv64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..2c896130a73 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c8f9bf468d6a56caeae53546d948fab7d54706d9c674dc5e943d94d3aa7390ffv64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c8f9bf468d6a56caeae53546d948fab7d54706d9c674dc5e943d94d3aa7390ffv64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c8f9bf468d6a56caeae53546d948fab7d54706d9c674dc5e943d94d3aa7390ffv64_0/Formula.class new file mode 100644 index 00000000000..e4eddfba2ec Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_c8f9bf468d6a56caeae53546d948fab7d54706d9c674dc5e943d94d3aa7390ffv64_0/Formula.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_d85e090333ad41d34f0b7335ffcf5c5a6fbf910bfbaab31f07bdeea0d64893aev64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_d85e090333ad41d34f0b7335ffcf5c5a6fbf910bfbaab31f07bdeea0d64893aev64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..8a9745e84a4 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_d85e090333ad41d34f0b7335ffcf5c5a6fbf910bfbaab31f07bdeea0d64893aev64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_d85e090333ad41d34f0b7335ffcf5c5a6fbf910bfbaab31f07bdeea0d64893aev64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_d85e090333ad41d34f0b7335ffcf5c5a6fbf910bfbaab31f07bdeea0d64893aev64_0/Formula.class new file mode 100644 index 00000000000..0f950f2e901 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_d85e090333ad41d34f0b7335ffcf5c5a6fbf910bfbaab31f07bdeea0d64893aev64_0/Formula.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_f0a36e4bf7dd41d038f9bd24522644fb6cedf543b97d594ef5ad5726bfe91cfev64_0/Formula$FormulaFillContext.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_f0a36e4bf7dd41d038f9bd24522644fb6cedf543b97d594ef5ad5726bfe91cfev64_0/Formula$FormulaFillContext.class new file mode 100644 index 00000000000..358924deb70 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_f0a36e4bf7dd41d038f9bd24522644fb6cedf543b97d594ef5ad5726bfe91cfev64_0/Formula$FormulaFillContext.class differ diff --git a/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_f0a36e4bf7dd41d038f9bd24522644fb6cedf543b97d594ef5ad5726bfe91cfev64_0/Formula.class b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_f0a36e4bf7dd41d038f9bd24522644fb6cedf543b97d594ef5ad5726bfe91cfev64_0/Formula.class new file mode 100644 index 00000000000..04c552f75e0 Binary files /dev/null and b/server/netty/io.deephaven.engine.context.QueryCompiler.createForUnitTests/io/deephaven/temp/c_f0a36e4bf7dd41d038f9bd24522644fb6cedf543b97d594ef5ad5726bfe91cfev64_0/Formula.class differ 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 a973910fa02..fefc8018e43 100644 --- a/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java +++ b/server/src/main/java/io/deephaven/server/barrage/BarrageMessageProducer.java @@ -364,7 +364,7 @@ public BarrageMessageProducer( parent.getColumnSourceMap().forEach((columnName, columnSource) -> { int ii = mi.getAndIncrement(); chunkWriters[ii] = DefaultChunkWriterFactory.INSTANCE.newWriter(BarrageTypeInfo.make( - ReinterpretUtils.maybeConvertToPrimitiveDataType(columnSource.getType()), + columnSource.getType(), columnSource.getComponentType(), schema.fields(ii))); }); 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 2e13cbb2fdb..2fd777fc828 100644 --- a/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableViewSubscription.java +++ b/server/src/main/java/io/deephaven/server/hierarchicaltable/HierarchicalTableViewSubscription.java @@ -365,7 +365,7 @@ private static void buildAndSendSnapshot( barrageMessage.addColumnData[ci] = addColumnData; chunkWriters[ci] = DefaultChunkWriterFactory.INSTANCE.newWriter(BarrageTypeInfo.make( - ReinterpretUtils.maybeConvertToPrimitiveDataType(columnDefinition.getDataType()), + columnDefinition.getDataType(), columnDefinition.getComponentType(), BarrageUtil.flatbufFieldFor(columnDefinition, Map.of()))); } diff --git a/server/src/main/java/io/deephaven/server/session/SessionState.java.orig b/server/src/main/java/io/deephaven/server/session/SessionState.java.orig new file mode 100644 index 00000000000..e45a86b33c6 --- /dev/null +++ b/server/src/main/java/io/deephaven/server/session/SessionState.java.orig @@ -0,0 +1,1558 @@ +/** + * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending + */ +package io.deephaven.server.session; + +import com.github.f4b6a3.uuid.UuidCreator; +import com.google.rpc.Code; +import dagger.assisted.Assisted; +import dagger.assisted.AssistedFactory; +import dagger.assisted.AssistedInject; +import io.deephaven.base.reference.WeakSimpleReference; +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.liveness.LivenessArtifact; +import io.deephaven.engine.liveness.LivenessReferent; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; +import io.deephaven.engine.table.impl.perf.QueryPerformanceRecorder; +import io.deephaven.engine.table.impl.perf.QueryState; +import io.deephaven.engine.table.impl.util.EngineMetrics; +import io.deephaven.engine.updategraph.DynamicNode; +import io.deephaven.hash.KeyedIntObjectHash; +import io.deephaven.hash.KeyedIntObjectHashMap; +import io.deephaven.hash.KeyedIntObjectKey; +import io.deephaven.internal.log.LoggerFactory; +import io.deephaven.io.log.LogEntry; +import io.deephaven.io.logger.Logger; +import io.deephaven.proto.backplane.grpc.ExportNotification; +import io.deephaven.proto.backplane.grpc.Ticket; +import io.deephaven.proto.flight.util.FlightExportTicketHelper; +import io.deephaven.proto.util.Exceptions; +import io.deephaven.proto.util.ExportTicketHelper; +import io.deephaven.server.util.Scheduler; +import io.deephaven.engine.context.ExecutionContext; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.annotations.VisibleForTesting; +import io.deephaven.auth.AuthContext; +import io.deephaven.util.datastructures.SimpleReferenceManager; +import io.deephaven.util.process.ProcessEnvironment; +import io.grpc.StatusRuntimeException; +import io.grpc.stub.StreamObserver; +import org.apache.arrow.flight.impl.Flight; +import org.apache.commons.lang3.mutable.MutableObject; +import org.jetbrains.annotations.NotNull; + +import javax.annotation.Nullable; +import javax.inject.Provider; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.Callable; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.function.Consumer; + +import static io.deephaven.base.log.LogOutput.MILLIS_FROM_EPOCH_FORMATTER; +import static io.deephaven.extensions.barrage.util.GrpcUtil.safelyComplete; +import static io.deephaven.extensions.barrage.util.GrpcUtil.safelyError; + +/** + * SessionState manages all exports for a single session. + * + *

+ * It manages exported {@link LivenessReferent}. It cascades failures to child dependencies. + * + *

+ * TODO: - cyclical dependency detection - out-of-order dependency timeout + * + *

+ * Details Regarding Data Structure of ExportObjects: + * + *

    + *
  • The exportMap map, exportListeners list, exportListenerVersion, and export object's exportListenerVersion work + * together to enable a listener to synchronize with outstanding exports in addition to sending the listener updates + * while they continue to subscribe.
  • + * + *
  • SessionState::exportMap's purpose is to map from the export id to the export object
  • + *
  • SessionState::exportListeners' purpose is to keep a list of active subscribers
  • + *
  • SessionState::exportListenerVersion's purpose is to know whether or not a subscriber has already seen a + * status
  • + * + *
  • A listener will receive an export notification for export id NON_EXPORT_ID (a zero) to indicate that the run has + * completed. A listener may see an update for an export before receiving the "run has completed" message. A listener + * should be prepared to receive duplicate/redundant updates.
  • + *
+ */ +public class SessionState { + // Some work items will be dependent on other exports, but do not export anything themselves. + public static final int NON_EXPORT_ID = 0; + + @AssistedFactory + public interface Factory { + SessionState create(AuthContext authContext); + } + + /** + * Wrap an object in an ExportObject to make it conform to the session export API. + * + * @param export the object to wrap + * @param the type of the object + * @return a sessionless export object + */ + public static ExportObject wrapAsExport(final T export) { + return new ExportObject<>(export); + } + + /** + * Wrap an exception in an ExportObject to make it conform to the session export API. The export behaves as if it + * has already failed. + * + * @param caughtException the exception to propagate + * @param the type of the object + * @return a sessionless export object + */ + public static ExportObject wrapAsFailedExport(final Exception caughtException) { + ExportObject exportObject = new ExportObject<>(null); + exportObject.caughtException = caughtException; + return exportObject; + } + + private static final Logger log = LoggerFactory.getLogger(SessionState.class); + + private final String logPrefix; + private final Scheduler scheduler; + private final SessionService.ErrorTransformer errorTransformer; + private final AuthContext authContext; + + private final String sessionId; + private volatile SessionService.TokenExpiration expiration = null; + private static final AtomicReferenceFieldUpdater EXPIRATION_UPDATER = + AtomicReferenceFieldUpdater.newUpdater(SessionState.class, SessionService.TokenExpiration.class, + "expiration"); + + // some types of exports have a more sound story if the server tells the client what to call it + private volatile int nextServerAllocatedId = -1; + private static final AtomicIntegerFieldUpdater SERVER_EXPORT_UPDATER = + AtomicIntegerFieldUpdater.newUpdater(SessionState.class, "nextServerAllocatedId"); + + // maintains all requested exports by this client's session + private final KeyedIntObjectHashMap> exportMap = new KeyedIntObjectHashMap<>(EXPORT_OBJECT_ID_KEY); + + // the list of active listeners + private final List exportListeners = new CopyOnWriteArrayList<>(); + private volatile int exportListenerVersion = 0; + + // Usually, export life cycles are managed explicitly with the life cycle of the session state. However, we need + // to be able to close non-exports that are not in the map but are otherwise satisfying outstanding gRPC requests. + private final SimpleReferenceManager> onCloseCallbacks = + new SimpleReferenceManager<>(WeakSimpleReference::new, false); + + private final ExecutionContext executionContext; + + @AssistedInject + public SessionState( + final Scheduler scheduler, + final SessionService.ErrorTransformer errorTransformer, + final Provider executionContextProvider, + @Assisted final AuthContext authContext) { + this.sessionId = UuidCreator.toString(UuidCreator.getRandomBased()); + this.logPrefix = "SessionState{" + sessionId + "}: "; + this.scheduler = scheduler; + this.errorTransformer = errorTransformer; + this.authContext = authContext; + this.executionContext = executionContextProvider.get().withAuthContext(authContext); + log.debug().append(logPrefix).append("session initialized").endl(); + } + + /** + * This method is controlled by SessionService to update the expiration whenever the session is refreshed. + * + * @param expiration the initial expiration time and session token + */ + @VisibleForTesting + protected void initializeExpiration(@NotNull final SessionService.TokenExpiration expiration) { + if (expiration.session != this) { + throw new IllegalArgumentException("mismatched session for expiration token"); + } + + if (!EXPIRATION_UPDATER.compareAndSet(this, null, expiration)) { + throw new IllegalStateException("session already initialized"); + } + + log.debug().append(logPrefix) + .append("token initialized to '").append(expiration.token.toString()) + .append("' which expires at ").append(MILLIS_FROM_EPOCH_FORMATTER, expiration.deadlineMillis) + .append(".").endl(); + } + + /** + * This method is controlled by SessionService to update the expiration whenever the session is refreshed. + * + * @param expiration the new expiration time and session token + */ + @VisibleForTesting + protected void updateExpiration(@NotNull final SessionService.TokenExpiration expiration) { + if (expiration.session != this) { + throw new IllegalArgumentException("mismatched session for expiration token"); + } + + SessionService.TokenExpiration prevToken = this.expiration; + while (prevToken != null) { + if (EXPIRATION_UPDATER.compareAndSet(this, prevToken, expiration)) { + break; + } + prevToken = this.expiration; + } + + if (prevToken == null) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + + log.debug().append(logPrefix).append("token, expires at ") + .append(MILLIS_FROM_EPOCH_FORMATTER, expiration.deadlineMillis).append(".").endl(); + } + + /** + * @return the session id + */ + public String getSessionId() { + return sessionId; + } + + /** + * @return the current expiration token for this session + */ + public SessionService.TokenExpiration getExpiration() { + if (isExpired()) { + return null; + } + return expiration; + } + + /** + * @return whether or not this session is expired + */ + public boolean isExpired() { + final SessionService.TokenExpiration currToken = expiration; + return currToken == null || currToken.deadlineMillis <= scheduler.currentTimeMillis(); + } + + /** + * @return the auth context for this session + */ + public AuthContext getAuthContext() { + return authContext; + } + + /** + * @return the execution context for this session + */ + public ExecutionContext getExecutionContext() { + return executionContext; + } + + /** + * Grab the ExportObject for the provided ticket. + * + * @param ticket the export ticket + * @param logId an end-user friendly identification of the ticket should an error occur + * @return a future-like object that represents this export + */ + public ExportObject getExport(final Ticket ticket, final String logId) { + return getExport(ExportTicketHelper.ticketToExportId(ticket, logId)); + } + + /** + * Grab the ExportObject for the provided ticket. + * + * @param ticket the export ticket + * @param logId an end-user friendly identification of the ticket should an error occur + * @return a future-like object that represents this export + */ + public ExportObject getExport(final Flight.Ticket ticket, final String logId) { + return getExport(FlightExportTicketHelper.ticketToExportId(ticket, logId)); + } + + /** + * Grab the ExportObject for the provided id. + * + * @param exportId the export handle id + * @return a future-like object that represents this export + */ + @SuppressWarnings("unchecked") + public ExportObject getExport(final int exportId) { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + + final ExportObject result; + + if (exportId < NON_EXPORT_ID) { + // If this a server-side export then it must already exist or else is a user error. + result = (ExportObject) exportMap.get(exportId); + + if (result == null) { + throw Exceptions.statusRuntimeException(Code.FAILED_PRECONDITION, + "Export id " + exportId + " does not exist and cannot be used out-of-order!"); + } + } else if (exportId > NON_EXPORT_ID) { + // If this a client-side export we'll allow an out-of-order request by creating a new export object. + result = (ExportObject) exportMap.putIfAbsent(exportId, EXPORT_OBJECT_VALUE_FACTORY); + } else { + // If this is a non-export request, then it is a user error. + throw Exceptions.statusRuntimeException(Code.INVALID_ARGUMENT, + "Export id " + exportId + " refers to a non-export and cannot be requested!"); + } + + return result; + } + + /** + * Grab the ExportObject for the provided id if it already exists, otherwise return null. + * + * @param exportId the export handle id + * @return a future-like object that represents this export + */ + @SuppressWarnings("unchecked") + public ExportObject getExportIfExists(final int exportId) { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + + return (ExportObject) exportMap.get(exportId); + } + + /** + * Grab the ExportObject for the provided id if it already exists, otherwise return null. + * + * @param ticket the export ticket + * @param logId an end-user friendly identification of the ticket should an error occur + * @return a future-like object that represents this export + */ + public ExportObject getExportIfExists(final Ticket ticket, final String logId) { + return getExportIfExists(ExportTicketHelper.ticketToExportId(ticket, logId)); + } + + /** + * Create and export a pre-computed element. This is typically used in scenarios where the number of exports is not + * known in advance by the requesting client. + * + * @param export the result of the export + * @param the export type + * @return the ExportObject for this item for ease of access to the export + */ + public ExportObject newServerSideExport(final T export) { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + + final int exportId = SERVER_EXPORT_UPDATER.getAndDecrement(this); + + // noinspection unchecked + final ExportObject result = (ExportObject) exportMap.putIfAbsent(exportId, EXPORT_OBJECT_VALUE_FACTORY); + result.setResult(export); + return result; + } + + /** + * Create an ExportBuilder to create the export after dependencies are satisfied. + * + * @param ticket the grpc {@link Flight.Ticket} for this export + * @param logId an end-user friendly identification of the ticket should an error occur + * @param the export type that the callable will return + * @return an export builder + */ + public ExportBuilder newExport(final Flight.Ticket ticket, final String logId) { + return newExport(FlightExportTicketHelper.ticketToExportId(ticket, logId)); + } + + /** + * Create an ExportBuilder to create the export after dependencies are satisfied. + * + * @param ticket the grpc {@link Ticket} for this export + * @param logId an end-user friendly identification of the ticket should an error occur + * @param the export type that the callable will return + * @return an export builder + */ + public ExportBuilder newExport(final Ticket ticket, final String logId) { + return newExport(ExportTicketHelper.ticketToExportId(ticket, logId)); + } + + /** + * Create an ExportBuilder to create the export after dependencies are satisfied. + * + * @param exportId the export id + * @param the export type that the callable will return + * @return an export builder + */ + @VisibleForTesting + public ExportBuilder newExport(final int exportId) { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + if (exportId <= 0) { + throw new IllegalArgumentException("exportId's <= 0 are reserved for server allocation only"); + } + return new ExportBuilder<>(exportId); + } + + /** + * Create an ExportBuilder to perform work after dependencies are satisfied that itself does not create any exports. + * + * @return an export builder + */ + public ExportBuilder nonExport() { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + return new ExportBuilder<>(NON_EXPORT_ID); + } + + /** + * Attach an on-close callback bound to the life of the session. Note that {@link Closeable} does not require that + * the close() method be idempotent, but when combined with {@link #removeOnCloseCallback(Closeable)}, close() will + * only be called once from this class. + *

+ *

+ * If called after the session has expired, this will throw, and the close() method on the provided instance will + * not be called. + * + * @param onClose the callback to invoke at end-of-life + */ + public void addOnCloseCallback(final Closeable onClose) { + synchronized (onCloseCallbacks) { + if (isExpired()) { + // After the session has expired, nothing new can be added to the collection, so throw an exception (and + // release the lock, allowing each item already in the collection to be released) + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + onCloseCallbacks.add(onClose); + } + } + + /** + * Remove an on-close callback bound to the life of the session. + *

+ * A common pattern to use this will be for an object to try to remove itself, and if it succeeds, to call its own + * {@link Closeable#close()}. If it fails, it can expect to have close() be called automatically. + * + * @param onClose the callback to no longer invoke at end-of-life + * @return true iff the callback was removed + * @apiNote If this SessionState has already begun expiration processing, {@code onClose} will not be removed by + * this method. This means that if {@code onClose} was previously added and not removed, it either has + * already been invoked or will be invoked by the SessionState. + */ + public boolean removeOnCloseCallback(final Closeable onClose) { + if (isExpired()) { + // After the session has expired, nothing can be removed from the collection. + return false; + } + synchronized (onCloseCallbacks) { + return onCloseCallbacks.remove(onClose) != null; + } + } + + /** + * Notes that this session has expired and exports should be released. + */ + public void onExpired() { + // note that once we set expiration to null; we are not able to add any more objects to the exportMap + SessionService.TokenExpiration prevToken = expiration; + while (prevToken != null) { + if (EXPIRATION_UPDATER.compareAndSet(this, prevToken, null)) { + break; + } + prevToken = expiration; + } + if (prevToken == null) { + // already expired + return; + } + + log.debug().append(logPrefix).append("releasing outstanding exports").endl(); + synchronized (exportMap) { + exportMap.forEach(ExportObject::cancel); + exportMap.clear(); + } + + log.debug().append(logPrefix).append("outstanding exports released").endl(); + synchronized (exportListeners) { + exportListeners.forEach(ExportListener::onRemove); + exportListeners.clear(); + } + + final List callbacksToClose; + synchronized (onCloseCallbacks) { + callbacksToClose = new ArrayList<>(onCloseCallbacks.size()); + onCloseCallbacks.forEach((ref, callback) -> callbacksToClose.add(callback)); + onCloseCallbacks.clear(); + } + callbacksToClose.forEach(callback -> { + try { + callback.close(); + } catch (final IOException e) { + log.error().append(logPrefix).append("error during onClose callback: ").append(e).endl(); + } + }); + } + + /** + * @return true iff the provided export state is a failure state + */ + public static boolean isExportStateFailure(final ExportNotification.State state) { + return state == ExportNotification.State.FAILED || state == ExportNotification.State.CANCELLED + || state == ExportNotification.State.DEPENDENCY_FAILED + || state == ExportNotification.State.DEPENDENCY_NEVER_FOUND + || state == ExportNotification.State.DEPENDENCY_RELEASED + || state == ExportNotification.State.DEPENDENCY_CANCELLED; + } + + /** + * @return true iff the provided export state is a terminal state + */ + public static boolean isExportStateTerminal(final ExportNotification.State state) { + return state == ExportNotification.State.RELEASED || isExportStateFailure(state); + } + + /** + * This class represents one unit of content exported in the session. + * + *

+ * Note: we reuse ExportObject for non-exporting tasks that have export dependencies. + * + * @param Is context-sensitive depending on the export. + * + * @apiNote ExportId may be 0, if this is a task that has exported dependencies, but does not export anything + * itself. Non-exports do not publish state changes. + */ + public final static class ExportObject extends LivenessArtifact { + private final int exportId; + private final String logIdentity; + private final SessionService.ErrorTransformer errorTransformer; + private final SessionState session; + + /** used to keep track of performance details either for aggregation or for the async ticket resolution */ + private QueryPerformanceRecorder queryPerformanceRecorder; + + /** final result of export */ + private volatile T result; + private volatile ExportNotification.State state = ExportNotification.State.UNKNOWN; + private volatile int exportListenerVersion = 0; + + /** Indicates whether this export has already been well defined. This prevents export object reuse. */ + private boolean hasHadWorkSet = false; + + /** This indicates whether or not this export should use the serial execution queue. */ + private boolean requiresSerialQueue; + + /** This is a reference of the work to-be-done. It is non-null only during the PENDING state. */ + private Callable exportMain; + /** This is a reference to the error handler to call if this item enters one of the failure states. */ + @Nullable + private ExportErrorHandler errorHandler; + /** This is a reference to the success handler to call if this item successfully exports. */ + @Nullable + private Consumer successHandler; + + /** used to keep track of which children need notification on export completion */ + private List> children = Collections.emptyList(); + /** used to manage liveness of dependencies (to prevent a dependency from being released before it is used) */ + private List> parents = Collections.emptyList(); + + /** used to detect when this object is ready for export (is visible for atomic int field updater) */ + private volatile int dependentCount = -1; + /** our first parent that was already released prior to having dependencies set if one exists */ + private ExportObject alreadyDeadParent; + + @SuppressWarnings("unchecked") + private static final AtomicIntegerFieldUpdater> DEPENDENT_COUNT_UPDATER = + AtomicIntegerFieldUpdater.newUpdater((Class>) (Class) ExportObject.class, + "dependentCount"); + + /** used to identify and propagate error details */ + private String errorId; + private String failedDependencyLogIdentity; + private Exception caughtException; + + /** + * @param errorTransformer the error transformer to use + * @param exportId the export id for this export + */ + private ExportObject( + final SessionService.ErrorTransformer errorTransformer, + final SessionState session, + final int exportId) { + super(true); + this.errorTransformer = errorTransformer; + this.session = session; + this.exportId = exportId; + this.logIdentity = + isNonExport() ? Integer.toHexString(System.identityHashCode(this)) : Long.toString(exportId); + setState(ExportNotification.State.UNKNOWN); + + // we retain a reference until a non-export becomes EXPORTED or a regular export becomes RELEASED + retainReference(); + } + + /** + * Create an ExportObject that is not tied to any session. These must be non-exports that have require no work + * to be performed. These export objects can be used as dependencies. + * + * @param result the object to wrap in an export + */ + private ExportObject(final T result) { + super(true); + this.errorTransformer = null; + this.session = null; + this.exportId = NON_EXPORT_ID; + this.result = result; + this.dependentCount = 0; + this.hasHadWorkSet = true; + this.logIdentity = Integer.toHexString(System.identityHashCode(this)) + "-sessionless"; + + if (result == null) { + maybeAssignErrorId(); + state = ExportNotification.State.FAILED; + } else { + state = ExportNotification.State.EXPORTED; + } + + if (result instanceof LivenessReferent && DynamicNode.notDynamicOrIsRefreshing(result)) { + manage((LivenessReferent) result); + } + } + + private boolean isNonExport() { + return exportId == NON_EXPORT_ID; + } + + private synchronized void setQueryPerformanceRecorder( + final QueryPerformanceRecorder queryPerformanceRecorder) { + if (this.queryPerformanceRecorder != null) { + throw new IllegalStateException( + "performance query recorder can only be set once on an exportable object"); + } + this.queryPerformanceRecorder = queryPerformanceRecorder; + } + + /** + * Sets the dependencies and tracks liveness dependencies. + * + * @param parents the dependencies that must be exported prior to invoking the exportMain callable + */ + private synchronized void setDependencies(final List> parents) { + if (dependentCount != -1) { + throw new IllegalStateException("dependencies can only be set once on an exportable object"); + } + + this.parents = parents; + dependentCount = parents.size(); + for (final ExportObject parent : parents) { + if (parent != null && !tryManage(parent)) { + // we've failed; let's cleanup already managed parents + forceReferenceCountToZero(); + alreadyDeadParent = parent; + break; + } + } + + if (log.isDebugEnabled()) { + final Exception e = new RuntimeException(); + final LogEntry entry = + log.debug().append(e).nl().append(session.logPrefix).append("export '").append(logIdentity) + .append("' has ").append(dependentCount).append(" dependencies remaining: "); + for (ExportObject parent : parents) { + entry.nl().append('\t').append(parent.logIdentity).append(" is ").append(parent.getState().name()); + } + entry.endl(); + } + } + + /** + * Sets the dependencies and initializes the relevant data structures to include this export as a child for + * each. + * + * @param exportMain the exportMain callable to invoke when dependencies are satisfied + * @param errorHandler the errorHandler to notify so that it may propagate errors to the requesting client + */ + private synchronized void setWork( + @NotNull final Callable exportMain, + @Nullable final ExportErrorHandler errorHandler, + @Nullable final Consumer successHandler, + final boolean requiresSerialQueue) { + if (hasHadWorkSet) { + throw new IllegalStateException("export object can only be defined once"); + } + hasHadWorkSet = true; + if (queryPerformanceRecorder != null && queryPerformanceRecorder.getState() == QueryState.RUNNING) { + // transfer ownership of the qpr to the export before it can be resumed by the scheduler + queryPerformanceRecorder.suspendQuery(); + } + this.requiresSerialQueue = requiresSerialQueue; + + // we defer this type of failure until setWork for consistency in error handling + if (alreadyDeadParent != null) { + onDependencyFailure(alreadyDeadParent); + alreadyDeadParent = null; + } + + if (isExportStateTerminal(state)) { + // The following scenarios cause us to get into this state: + // - this export object was released/cancelled + // - the session expiration propagated to this export object + // - a parent export was released/dead prior to `setDependencies` + // Note that failed dependencies will be handled in the onResolveOne method below. + + // since this is the first we know of the errorHandler, it could not have been invoked yet + if (errorHandler != null) { + maybeAssignErrorId(); + errorHandler.onError(state, errorId, caughtException, failedDependencyLogIdentity); + } + return; + } + + this.exportMain = exportMain; + this.errorHandler = errorHandler; + this.successHandler = successHandler; + + if (state != ExportNotification.State.PUBLISHING) { + setState(ExportNotification.State.PENDING); + } else if (dependentCount > 0) { + throw new IllegalStateException("published exports cannot have dependencies"); + } + if (dependentCount <= 0) { + dependentCount = 0; + scheduleExport(); + } else { + for (final ExportObject parent : parents) { + // we allow parents to be null to simplify calling conventions around optional dependencies + if (parent == null || !parent.maybeAddDependency(this)) { + onResolveOne(parent); + } + // else parent will notify us on completion + } + } + } + + /** + * WARNING! This method call is only safe to use in the following patterns: + *

+ * 1) If an export (or non-export) {@link ExportBuilder#require}'d this export then the method is valid from + * within the Callable/Runnable passed to {@link ExportBuilder#submit}. + *

+ * 2) By first obtaining a reference to the {@link ExportObject}, and then observing its state as + * {@link ExportNotification.State#EXPORTED}. The caller must abide by the Liveness API and dropReference. + *

+ * Example: + * + *

+         * {@code
+         *  T getFromExport(ExportObject export) {
+         *     if (export.tryRetainReference()) {
+         *         try {
+         *             if (export.getState() == ExportNotification.State.EXPORTED) {
+         *                 return export.get();
+         *             }
+         *         } finally {
+         *             export.dropReference();
+         *         }
+         *     }
+         *     return null;
+         * }
+         * }
+         * 
+ * + * @return the result of the computed export + */ + public T get() { + if (session != null && session.isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + final T localResult = result; + // Note: an export may be released while still being a dependency of queued work; so let's make sure we're + // still valid + if (localResult == null) { + throw new IllegalStateException( + "Dependent export '" + exportId + "' is null and in state " + state.name()); + } + return localResult; + } + + /** + * @return the current state of this export + */ + public ExportNotification.State getState() { + return state; + } + + /** + * @return the ticket for this export; note if this is a non-export the returned ticket will not resolve to + * anything and is considered an invalid ticket + */ + public Ticket getExportId() { + return ExportTicketHelper.wrapExportIdInTicket(exportId); + } + + /** + * Add dependency if object export has not yet completed. + * + * @param child the dependent task + * @return true if the child was added as a dependency + */ + private boolean maybeAddDependency(final ExportObject child) { + if (state == ExportNotification.State.EXPORTED || isExportStateTerminal(state)) { + return false; + } + synchronized (this) { + if (state == ExportNotification.State.EXPORTED || isExportStateTerminal(state)) { + return false; + } + + if (children.isEmpty()) { + children = new ArrayList<>(); + } + children.add(child); + return true; + } + } + + /** + * This helper notifies any export notification listeners, and propagates resolution to children that depend on + * this export. + * + * @param state the new state for this export + */ + private synchronized void setState(final ExportNotification.State state) { + if ((this.state == ExportNotification.State.EXPORTED && isNonExport()) + || isExportStateTerminal(this.state)) { + throw new IllegalStateException("cannot change state if export is already in terminal state"); + } + if (this.state != ExportNotification.State.UNKNOWN && this.state.getNumber() >= state.getNumber()) { + throw new IllegalStateException("export object state changes must advance toward a terminal state"); + } + this.state = state; + + // Send an export notification before possibly notifying children of our state change. + if (exportId != NON_EXPORT_ID) { + log.debug().append(session.logPrefix).append("export '").append(logIdentity) + .append("' is ExportState.").append(state.name()).endl(); + + final ExportNotification notification = makeExportNotification(); + exportListenerVersion = session.exportListenerVersion; + session.exportListeners.forEach(listener -> listener.notify(notification)); + } else { + log.debug().append(session == null ? "Session " : session.logPrefix) + .append("non-export '").append(logIdentity).append("' is ExportState.") + .append(state.name()).endl(); + } + + if (isExportStateFailure(state) && errorHandler != null) { + maybeAssignErrorId(); + try { + final Exception toReport; + if (caughtException != null && errorTransformer != null) { + toReport = errorTransformer.transform(caughtException); + } else { + toReport = caughtException; + } + + errorHandler.onError(state, errorId, toReport, failedDependencyLogIdentity); + } catch (final Throwable err) { + // this is a serious error; crash the jvm to ensure that we don't miss it + log.error().append("Unexpected error while reporting ExportObject failure: ").append(err).endl(); + ProcessEnvironment.getGlobalFatalErrorReporter().reportAsync( + "Unexpected error while reporting ExportObject failure", err); + } + } + + final boolean isNowExported = state == ExportNotification.State.EXPORTED; + if (isNowExported && successHandler != null) { + try { + successHandler.accept(result); + } catch (final Throwable err) { + // this is a serious error; crash the jvm to ensure that we don't miss it + log.error().append("Unexpected error while reporting ExportObject success: ").append(err).endl(); + ProcessEnvironment.getGlobalFatalErrorReporter().reportAsync( + "Unexpected error while reporting ExportObject success", err); + } + } + + if (isNowExported || isExportStateTerminal(state)) { + children.forEach(child -> child.onResolveOne(this)); + children = Collections.emptyList(); + parents.stream().filter(Objects::nonNull).forEach(this::tryUnmanage); + parents = Collections.emptyList(); + exportMain = null; + errorHandler = null; + successHandler = null; + } + + if ((isNowExported && isNonExport()) || isExportStateTerminal(state)) { + dropReference(); + } + } + + /** + * Decrements parent counter and kicks off the export if that was the last dependency. + * + * @param parent the parent that just resolved; it may have failed + */ + private void onResolveOne(@Nullable final ExportObject parent) { + // am I already cancelled or failed? + if (isExportStateTerminal(state)) { + return; + } + + // Is this a cascading failure? Note that we manage the parents in `setDependencies` which + // keeps the parent results live until this child been exported. This means that the parent is allowed to + // be in a RELEASED state, but is not allowed to be in a failure state. + if (parent != null && isExportStateFailure(parent.state)) { + onDependencyFailure(parent); + return; + } + + final int newDepCount = DEPENDENT_COUNT_UPDATER.decrementAndGet(this); + if (newDepCount > 0) { + return; // either more dependencies to wait for or this export has already failed + } + Assert.eqZero(newDepCount, "newDepCount"); + + scheduleExport(); + } + + /** + * Schedules the export to be performed; assumes all dependencies have been resolved. + */ + private void scheduleExport() { + synchronized (this) { + if (state != ExportNotification.State.PENDING && state != ExportNotification.State.PUBLISHING) { + return; + } + setState(ExportNotification.State.QUEUED); + } + + if (requiresSerialQueue) { + session.scheduler.runSerially(this::doExport); + } else { + session.scheduler.runImmediately(this::doExport); + } + } + + /** + * Performs the actual export on a scheduling thread. + */ + private void doExport() { + final Callable capturedExport; + synchronized (this) { + capturedExport = exportMain; + // check for some sort of cancel race with client + if (state != ExportNotification.State.QUEUED + || session.isExpired() + || capturedExport == null + || !tryRetainReference()) { + if (!isExportStateTerminal(state)) { + setState(ExportNotification.State.CANCELLED); + } else if (errorHandler != null) { + // noinspection ThrowableNotThrown + Assert.statementNeverExecuted("in terminal state but error handler is not null"); + } + return; + } + dropReference(); + setState(ExportNotification.State.RUNNING); + } + + T localResult = null; + boolean shouldLog = false; + final QueryPerformanceRecorder exportRecorder; + try (final SafeCloseable ignored1 = session.executionContext.open(); + final SafeCloseable ignored2 = LivenessScopeStack.open()) { + + final String queryId; + if (isNonExport()) { + queryId = "nonExport=" + logIdentity; + } else { + queryId = "exportId=" + logIdentity; + } + + final boolean isResume = queryPerformanceRecorder != null + && queryPerformanceRecorder.getState() == QueryState.SUSPENDED; + exportRecorder = Objects.requireNonNullElseGet(queryPerformanceRecorder, + () -> QueryPerformanceRecorder.newQuery("ExportObject#doWork(" + queryId + ")", + session.getSessionId(), QueryPerformanceNugget.DEFAULT_FACTORY)); + + try (final SafeCloseable ignored3 = isResume + ? exportRecorder.resumeQuery() + : exportRecorder.startQuery()) { + try { + localResult = capturedExport.call(); + } catch (final Exception err) { + caughtException = err; + } + shouldLog = exportRecorder.endQuery(); + } catch (final Exception err) { + // end query will throw if the export runner left the QPR in a bad state + if (caughtException == null) { + caughtException = err; + } + } + + if (caughtException != null) { + synchronized (this) { + if (!isExportStateTerminal(state)) { + maybeAssignErrorId(); + if (!(caughtException instanceof StatusRuntimeException)) { + log.error().append("Internal Error '").append(errorId).append("' ") + .append(caughtException).endl(); + } + setState(ExportNotification.State.FAILED); + } + } + } + if (shouldLog || caughtException != null) { + EngineMetrics.getInstance().logQueryProcessingResults(exportRecorder, caughtException); + } + if (caughtException == null) { + // must set result after ending the query so that onSuccess may resume / finalize a parent query + setResult(localResult); + } + } + } + + private void maybeAssignErrorId() { + if (errorId == null) { + errorId = UuidCreator.toString(UuidCreator.getRandomBased()); + } + } + + private synchronized void onDependencyFailure(final ExportObject parent) { + errorId = parent.errorId; + if (parent.caughtException instanceof StatusRuntimeException) { + caughtException = parent.caughtException; + } + ExportNotification.State terminalState = ExportNotification.State.DEPENDENCY_FAILED; + + if (errorId == null) { + final String errorDetails; + switch (parent.state) { + case RELEASED: + terminalState = ExportNotification.State.DEPENDENCY_RELEASED; + errorDetails = "dependency released by user."; + break; + case CANCELLED: + terminalState = ExportNotification.State.DEPENDENCY_CANCELLED; + errorDetails = "dependency cancelled by user."; + break; + default: + // Note: the other error states should have non-null errorId + errorDetails = "dependency does not have its own error defined " + + "and is in an unexpected state: " + parent.state; + break; + } + + maybeAssignErrorId(); + failedDependencyLogIdentity = parent.logIdentity; + if (!(caughtException instanceof StatusRuntimeException)) { + log.error().append("Internal Error '").append(errorId).append("' ").append(errorDetails) + .endl(); + } + } + + setState(terminalState); + } + + /** + * Sets the final result for this export. + * + * @param result the export object + */ + private void setResult(final T result) { + if (this.result != null) { + throw new IllegalStateException("cannot setResult twice!"); + } + + // result is cleared on destroy; so don't set if it won't be called + if (!tryRetainReference()) { + return; + } + + try { + synchronized (this) { + // client may race a cancel with setResult + if (!isExportStateTerminal(state)) { + this.result = result; + if (result instanceof LivenessReferent && DynamicNode.notDynamicOrIsRefreshing(result)) { + manage((LivenessReferent) result); + } + setState(ExportNotification.State.EXPORTED); + } + } + } finally { + dropReference(); + } + } + + /** + * Releases this export; it will wait for the work to complete before releasing. + */ + public synchronized void release() { + if (session == null) { + throw new UnsupportedOperationException("Session-less exports cannot be released"); + } + if (state == ExportNotification.State.EXPORTED) { + if (isNonExport()) { + return; + } + setState(ExportNotification.State.RELEASED); + } else if (!isExportStateTerminal(state)) { + session.nonExport().require(this).submit(this::release); + } + } + + /** + * Releases this export; it will cancel the work and dependent exports proactively when possible. + */ + public synchronized void cancel() { + if (session == null) { + throw new UnsupportedOperationException("Session-less exports cannot be cancelled"); + } + if (state == ExportNotification.State.EXPORTED) { + if (isNonExport()) { + return; + } + setState(ExportNotification.State.RELEASED); + } else if (!isExportStateTerminal(state)) { + setState(ExportNotification.State.CANCELLED); + } + } + + @Override + protected synchronized void destroy() { + super.destroy(); + result = null; + // keep SREs since error propagation won't reference a real errorId on the server + if (!(caughtException instanceof StatusRuntimeException)) { + caughtException = null; + } + } + + /** + * @return an export notification representing current state + */ + private synchronized ExportNotification makeExportNotification() { + final ExportNotification.Builder builder = ExportNotification.newBuilder() + .setTicket(ExportTicketHelper.wrapExportIdInTicket(exportId)) + .setExportState(state); + + if (errorId != null) { + builder.setContext(errorId); + } + if (failedDependencyLogIdentity != null) { + builder.setDependentHandle(failedDependencyLogIdentity); + } + + return builder.build(); + } + } + + public void addExportListener(final StreamObserver observer) { + final int versionId; + final ExportListener listener; + synchronized (exportListeners) { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + + listener = new ExportListener(observer); + exportListeners.add(listener); + versionId = ++exportListenerVersion; + } + + listener.initialize(versionId); + } + + /** + * Remove an on-close callback bound to the life of the session. + * + * @param observer the observer to no longer be subscribed + * @return The item if it was removed, else null + */ + public StreamObserver removeExportListener(final StreamObserver observer) { + final MutableObject wrappedListener = new MutableObject<>(); + final boolean found = exportListeners.removeIf(wrap -> { + if (wrappedListener.getValue() != null) { + return false; + } + + final boolean matches = wrap.listener == observer; + if (matches) { + wrappedListener.setValue(wrap); + } + return matches; + }); + + if (found) { + wrappedListener.getValue().onRemove(); + } + + return found ? observer : null; + } + + @VisibleForTesting + public long numExportListeners() { + return exportListeners.size(); + } + + private class ExportListener { + private volatile boolean isClosed = false; + + private final StreamObserver listener; + + private ExportListener(final StreamObserver listener) { + this.listener = listener; + } + + /** + * Propagate the change to the listener. + * + * @param notification the notification to send + */ + public void notify(final ExportNotification notification) { + if (isClosed) { + return; + } + + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + synchronized (listener) { + listener.onNext(notification); + } + } catch (final RuntimeException e) { + log.error().append("Failed to notify listener: ").append(e).endl(); + removeExportListener(listener); + } + } + + /** + * Perform the run and send initial export state to the listener. + */ + private void initialize(final int versionId) { + final String id = Integer.toHexString(System.identityHashCode(this)); + log.debug().append(logPrefix).append("refreshing listener ").append(id).endl(); + + for (final ExportObject export : exportMap) { + if (!export.tryRetainReference()) { + continue; + } + + try { + if (export.exportListenerVersion >= versionId) { + continue; + } + + // the export cannot change state while we are synchronized on it + // noinspection SynchronizationOnLocalVariableOrMethodParameter + synchronized (export) { + // check again because of race to the lock + if (export.exportListenerVersion >= versionId) { + continue; + } + + // no need to notify on exports that can no longer be accessed + if (isExportStateTerminal(export.getState())) { + continue; + } + + notify(export.makeExportNotification()); + } + } finally { + export.dropReference(); + } + } + + // notify that the run has completed + notify(ExportNotification.newBuilder() + .setTicket(ExportTicketHelper.wrapExportIdInTicket(NON_EXPORT_ID)) + .setExportState(ExportNotification.State.EXPORTED) + .setContext("run is complete") + .build()); + log.debug().append(logPrefix).append("run complete for listener ").append(id).endl(); + } + + protected void onRemove() { + synchronized (this) { + if (isClosed) { + return; + } + isClosed = true; + } + + safelyComplete(listener); + } + } + + @FunctionalInterface + public interface ExportErrorHandler { + /** + * Notify the handler that the final state of this export failed. + * + * @param resultState the final state of the export + * @param errorContext an identifier to locate the details as to why the export failed + * @param dependentExportId an identifier for the export id of the dependent that caused the failure if + * applicable + */ + void onError(final ExportNotification.State resultState, + final String errorContext, + @Nullable final Exception cause, + @Nullable final String dependentExportId); + } + @FunctionalInterface + public interface ExportErrorGrpcHandler { + /** + * This error handler receives a grpc friendly {@link StatusRuntimeException} that can be directly sent to + * {@link StreamObserver#onError}. + * + * @param notification the notification to forward to the grpc client + */ + void onError(final StatusRuntimeException notification); + } + + public class ExportBuilder { + private final int exportId; + private final ExportObject export; + + private boolean requiresSerialQueue; + private ExportErrorHandler errorHandler; + private Consumer successHandler; + + ExportBuilder(final int exportId) { + this.exportId = exportId; + + if (exportId == NON_EXPORT_ID) { + this.export = new ExportObject<>(SessionState.this.errorTransformer, SessionState.this, NON_EXPORT_ID); + } else { + // noinspection unchecked + this.export = (ExportObject) exportMap.putIfAbsent(exportId, EXPORT_OBJECT_VALUE_FACTORY); + } + } + + /** + * Set the performance recorder to resume when running this export. + * + * @param queryPerformanceRecorder the performance recorder + * @return this builder + */ + public ExportBuilder queryPerformanceRecorder( + @NotNull final QueryPerformanceRecorder queryPerformanceRecorder) { + export.setQueryPerformanceRecorder(queryPerformanceRecorder); + return this; + } + + /** + * Some exports must happen serially w.r.t. other exports. For example, an export that acquires the exclusive + * UGP lock. We enqueue these dependencies independently of the otherwise regularly concurrent exports. + * + * @return this builder + */ + public ExportBuilder requiresSerialQueue() { + requiresSerialQueue = true; + return this; + } + + /** + * Invoke this method to set the required dependencies for this export. A parent may be null to simplify usage + * of optional export dependencies. + * + * @param dependencies the parent dependencies + * @return this builder + */ + public ExportBuilder require(final ExportObject... dependencies) { + export.setDependencies(List.of(dependencies)); + return this; + } + + /** + * Invoke this method to set the required dependencies for this export. A parent may be null to simplify usage + * of optional export dependencies. + * + * @param dependencies the parent dependencies + * @return this builder + */ + public ExportBuilder require(final List> dependencies) { + export.setDependencies(List.copyOf(dependencies)); + return this; + } + + /** + * Invoke this method to set the error handler to be notified if this export fails. Only one error handler may + * be set. Exactly one of the onError and onSuccess handlers will be invoked. + *

+ * Not synchronized, it is expected that the provided callback handles thread safety itself. + * + * @param errorHandler the error handler to be notified + * @return this builder + */ + public ExportBuilder onError(final ExportErrorHandler errorHandler) { + if (this.errorHandler != null) { + throw new IllegalStateException("error handler already set"); + } else if (export.hasHadWorkSet) { + throw new IllegalStateException("error handler must be set before work is submitted"); + } + this.errorHandler = errorHandler; + return this; + } + + /** + * Invoke this method to set the error handler to be notified if this export fails. Only one error handler may + * be set. Exactly one of the onError and onSuccess handlers will be invoked. + *

+ * Not synchronized, it is expected that the provided callback handles thread safety itself. + * + * @param errorHandler the error handler to be notified + * @return this builder + */ + public ExportBuilder onErrorHandler(final ExportErrorGrpcHandler errorHandler) { + return onError(((resultState, errorContext, cause, dependentExportId) -> { + if (cause instanceof StatusRuntimeException) { + errorHandler.onError((StatusRuntimeException) cause); + return; + } + + final String dependentStr = dependentExportId == null ? "" + : (" (related parent export id: " + dependentExportId + ")"); + if (cause == null) { + if (resultState == ExportNotification.State.CANCELLED) { + errorHandler.onError(Exceptions.statusRuntimeException(Code.CANCELLED, + "Export is cancelled" + dependentStr)); + } else { + errorHandler.onError(Exceptions.statusRuntimeException(Code.FAILED_PRECONDITION, + "Export in state " + resultState + dependentStr)); + } + } else { + errorHandler.onError(Exceptions.statusRuntimeException(Code.FAILED_PRECONDITION, + "Details Logged w/ID '" + errorContext + "'" + dependentStr)); + } + })); + } + + /** + * Invoke this method to set the error handler to be notified if this export fails. Only one error handler may + * be set. This is a convenience method for use with {@link StreamObserver}. Exactly one of the onError and + * onSuccess handlers will be invoked. + *

+ * Invoking onError will be synchronized on the StreamObserver instance, so callers can rely on that mechanism + * to deal with more than one thread trying to write to the stream. + * + * @param streamObserver the streamObserver to be notified of any error + * @return this builder + */ + public ExportBuilder onError(StreamObserver streamObserver) { + return onErrorHandler(statusRuntimeException -> { + safelyError(streamObserver, statusRuntimeException); + }); + } + + /** + * Invoke this method to set the onSuccess handler to be notified if this export succeeds. Only one success + * handler may be set. Exactly one of the onError and onSuccess handlers will be invoked. + *

+ * Not synchronized, it is expected that the provided callback handles thread safety itself. + * + * @param successHandler the onSuccess handler to be notified + * @return this builder + */ + public ExportBuilder onSuccess(final Consumer successHandler) { + if (this.successHandler != null) { + throw new IllegalStateException("success handler already set"); + } else if (export.hasHadWorkSet) { + throw new IllegalStateException("success handler must be set before work is submitted"); + } + this.successHandler = successHandler; + return this; + } + + /** + * Invoke this method to set the onSuccess handler to be notified if this export succeeds. Only one success + * handler may be set. Exactly one of the onError and onSuccess handlers will be invoked. + *

+ * Not synchronized, it is expected that the provided callback handles thread safety itself. + * + * @param successHandler the onSuccess handler to be notified + * @return this builder + */ + public ExportBuilder onSuccess(final Runnable successHandler) { + return onSuccess(ignored -> successHandler.run()); + } + + /** + * This method is the final method for submitting an export to the session. The provided callable is enqueued on + * the scheduler when all dependencies have been satisfied. Only the dependencies supplied to the builder are + * guaranteed to be resolved when the exportMain is executing. + *

+ * Warning! It is the SessionState owner's responsibility to wait to release any dependency until after this + * exportMain callable/runnable has complete. + * + * @param exportMain the callable that generates the export + * @return the submitted export object + */ + public ExportObject submit(final Callable exportMain) { + export.setWork(exportMain, errorHandler, successHandler, requiresSerialQueue); + return export; + } + + /** + * This method is the final method for submitting an export to the session. The provided runnable is enqueued on + * the scheduler when all dependencies have been satisfied. Only the dependencies supplied to the builder are + * guaranteed to be resolved when the exportMain is executing. + *

+ * Warning! It is the SessionState owner's responsibility to wait to release any dependency until after this + * exportMain callable/runnable has complete. + * + * @param exportMain the runnable to execute once dependencies have resolved + * @return the submitted export object + */ + public ExportObject submit(final Runnable exportMain) { + return submit(() -> { + exportMain.run(); + return null; + }); + } + + /** + * @return the export object that this builder is building + */ + public ExportObject getExport() { + return export; + } + + /** + * @return the export id of this export or {@link SessionState#NON_EXPORT_ID} if is a non-export + */ + public int getExportId() { + return exportId; + } + } + + private static final KeyedIntObjectKey> EXPORT_OBJECT_ID_KEY = + new KeyedIntObjectKey.BasicStrict>() { + @Override + public int getIntKey(final ExportObject exportObject) { + return exportObject.exportId; + } + }; + + private final KeyedIntObjectHash.ValueFactory> EXPORT_OBJECT_VALUE_FACTORY = + new KeyedIntObjectHash.ValueFactory.Strict>() { + @Override + public ExportObject newValue(final int key) { + if (isExpired()) { + throw Exceptions.statusRuntimeException(Code.UNAUTHENTICATED, "session has expired"); + } + + return new ExportObject<>(SessionState.this.errorTransformer, SessionState.this, key); + } + }; +} 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 6ac5505241f..69db8c8c5c5 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 @@ -16,7 +16,6 @@ import io.deephaven.barrage.flatbuf.BarrageMessageWrapper; import io.deephaven.barrage.flatbuf.BarrageSnapshotOptions; import io.deephaven.barrage.flatbuf.BarrageSnapshotRequest; -import io.deephaven.barrage.flatbuf.ColumnConversionMode; import io.deephaven.base.clock.Clock; import io.deephaven.base.verify.Assert; import io.deephaven.client.impl.*; @@ -1116,7 +1115,6 @@ private void testLongColumnWithFactor(org.apache.arrow.vector.types.TimeUnit tim for (int ii = 0; ii < numRows; ++ii) { vector.set(ii, ii % 3 == 0 ? QueryConstants.NULL_LONG : ii); } - vector.setValueCount(numRows); root.setRowCount(numRows); stream.putNext(); @@ -1171,7 +1169,6 @@ private void testInstantColumnWithFactor( for (int ii = 0; ii < numRows; ++ii) { vector.set(ii, ii % 3 == 0 ? QueryConstants.NULL_LONG : ii); } - vector.setValueCount(numRows); root.setRowCount(numRows); stream.putNext(); @@ -1225,7 +1222,6 @@ private void testZonedDateTimeColumnWithFactor( for (int ii = 0; ii < numRows; ++ii) { vector.set(ii, ii % 3 == 0 ? QueryConstants.NULL_LONG : ii); } - vector.setValueCount(numRows); root.setRowCount(numRows); stream.putNext(); @@ -1272,12 +1268,12 @@ public void testNullNestedPrimitiveArray() { final FlightClient.ClientStreamListener stream = flightClient.startPut( FlightDescriptor.path("export", Integer.toString(exportId)), root, new SyncPutListener()); + final int numRows = 1; outerVector.allocateNew(); - UnionListWriter listWriter = new UnionListWriter(outerVector); - final int numRows = 1; + final UnionListWriter listWriter = outerVector.getWriter(); listWriter.writeNull(); - listWriter.setValueCount(numRows); + root.setRowCount(numRows); stream.putNext(); @@ -1305,13 +1301,12 @@ public void testEmptyNestedPrimitiveArray() { final FlightClient.ClientStreamListener stream = flightClient.startPut( FlightDescriptor.path("export", Integer.toString(exportId)), root, new SyncPutListener()); + final int numRows = 1; outerVector.allocateNew(); - UnionListWriter listWriter = new UnionListWriter(outerVector); + final UnionListWriter listWriter = outerVector.getWriter(); - final int numRows = 1; listWriter.startList(); listWriter.endList(); - listWriter.setValueCount(0); root.setRowCount(numRows); @@ -1342,15 +1337,15 @@ public void testInterestingNestedPrimitiveArray() { final FlightClient.ClientStreamListener stream = flightClient.startPut( FlightDescriptor.path("export", Integer.toString(exportId)), root, new SyncPutListener()); + final int numRows = 1; outerVector.allocateNew(); - UnionListWriter listWriter = new UnionListWriter(outerVector); + final UnionListWriter listWriter = outerVector.getWriter(); - final int numRows = 1; // We want to recreate this structure: // new double[][] { null, new double[] {}, new double[] { 42.42f, 43.43f } } listWriter.startList(); - BaseWriter.ListWriter innerListWriter = listWriter.list(); + final BaseWriter.ListWriter innerListWriter = listWriter.list(); // null inner list innerListWriter.writeNull(); @@ -1366,7 +1361,6 @@ public void testInterestingNestedPrimitiveArray() { innerListWriter.endList(); listWriter.endList(); - listWriter.setValueCount(numRows); root.setRowCount(numRows); stream.putNext(); 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 f1b037854fd..5821ade2fe7 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 @@ -275,7 +275,8 @@ public > ChunkReader newReader( final ExpansionKernel kernel = ArrayExpansionKernel.makeExpansionKernel(chunkType, componentTypeInfo.type()); final ChunkReader componentReader = newReader(componentTypeInfo, options); - return (ChunkReader) new ListChunkReader<>(ListChunkReader.Mode.DENSE, 0, kernel, componentReader); + return (ChunkReader) new ListChunkReader<>(ListChunkReader.Mode.VARIABLE, 0, kernel, + componentReader); } default: throw new IllegalArgumentException("Unsupported type: " + Type.name(typeInfo.arrowField().typeType()));