From 4d3947bc980289a1889f3d0c35c5c51b9a270cb3 Mon Sep 17 00:00:00 2001 From: Colin Alworth Date: Fri, 9 Feb 2024 11:30:49 -0600 Subject: [PATCH] Replace usage of trove in Chunk APIs (#5058) Partial #188 --- .../BooleanChunkInputStreamGenerator.java | 10 +++---- .../chunk/ByteChunkInputStreamGenerator.java | 10 +++---- .../chunk/CharChunkInputStreamGenerator.java | 10 +++---- .../chunk/ChunkInputStreamGenerator.java | 7 +++-- .../DoubleChunkInputStreamGenerator.java | 10 +++---- .../FixedWidthChunkInputStreamGenerator.java | 8 +++--- .../chunk/FloatChunkInputStreamGenerator.java | 10 +++---- .../chunk/IntChunkInputStreamGenerator.java | 10 +++---- .../chunk/LongChunkInputStreamGenerator.java | 10 +++---- .../chunk/ShortChunkInputStreamGenerator.java | 10 +++---- .../VarBinaryChunkInputStreamGenerator.java | 10 +++---- .../VarListChunkInputStreamGenerator.java | 8 +++--- .../VectorChunkInputStreamGenerator.java | 8 +++--- .../barrage/util/ArrowToTableConverter.java | 10 +++---- .../util/BarrageChunkAppendingMarshaller.java | 10 +++---- .../barrage/util/BarrageStreamReader.java | 10 +++---- .../chunk/BarrageColumnRoundTripTest.java | 26 ++++++++++++------- 17 files changed, 91 insertions(+), 86 deletions(-) diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java index ce247ef8bd0..b0d5bc4c49a 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/BooleanChunkInputStreamGenerator.java @@ -3,7 +3,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -23,6 +22,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -162,7 +162,7 @@ public interface ByteConversion { static WritableChunk extractChunkFromInputStream( final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -175,15 +175,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final ByteConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableByteChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java index e341c37c8eb..322d32fc468 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ByteChunkInputStreamGenerator.java @@ -8,7 +8,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -165,7 +165,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -179,15 +179,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final ByteConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableByteChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java index cdd9e0a53fe..fcc631e674b 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/CharChunkInputStreamGenerator.java @@ -3,7 +3,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -23,6 +22,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -160,7 +160,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -174,15 +174,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final CharConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableCharChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java index c7591daaa92..9674f4f4c54 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ChunkInputStreamGenerator.java @@ -4,7 +4,6 @@ package io.deephaven.extensions.barrage.chunk; import com.google.common.base.Charsets; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableLongChunk; @@ -28,10 +27,10 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.time.Instant; -import java.time.ZoneId; import java.time.ZonedDateTime; import java.util.Arrays; import java.util.Iterator; +import java.util.PrimitiveIterator; public interface ChunkInputStreamGenerator extends SafeCloseable { @@ -158,7 +157,7 @@ static WritableChunk extractChunkFromInputStream( final StreamReaderOptions options, final ChunkType chunkType, final Class type, final Class componentType, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int offset, final int totalRows) throws IOException { return extractChunkFromInputStream(options, 1, chunkType, type, componentType, fieldNodeIter, bufferInfoIter, is, @@ -170,7 +169,7 @@ static WritableChunk extractChunkFromInputStream( final int factor, final ChunkType chunkType, final Class type, final Class componentType, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { switch (chunkType) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java index 299cf482eb6..8aea823b31f 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/DoubleChunkInputStreamGenerator.java @@ -8,7 +8,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -165,7 +165,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -179,15 +179,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final DoubleConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableDoubleChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java index 14c16b02c4e..3cbbaef01ac 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FixedWidthChunkInputStreamGenerator.java @@ -3,7 +3,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableObjectChunk; @@ -14,6 +13,7 @@ import java.io.DataInput; import java.io.IOException; import java.util.Iterator; +import java.util.PrimitiveIterator; public class FixedWidthChunkInputStreamGenerator { private static final String DEBUG_NAME = "FixedWidthChunkInputStreamGenerator"; @@ -45,15 +45,15 @@ static WritableObjectChunk extractChunkFromInputStreamWithTypeCon final StreamReaderOptions options, final TypeConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final ChunkInputStreamGenerator.FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableObjectChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java index f730cf77bb9..2b8b99d24d8 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/FloatChunkInputStreamGenerator.java @@ -8,7 +8,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -165,7 +165,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -179,15 +179,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final FloatConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableFloatChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java index f3f8ee749ad..31ef529f840 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/IntChunkInputStreamGenerator.java @@ -8,7 +8,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -165,7 +165,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -179,15 +179,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final IntConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableIntChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java index 70480bdb702..244c606c38e 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/LongChunkInputStreamGenerator.java @@ -8,7 +8,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -165,7 +165,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -179,15 +179,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final LongConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableLongChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java index e27167629e8..5da4e1ecea8 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/ShortChunkInputStreamGenerator.java @@ -8,7 +8,6 @@ */ package io.deephaven.extensions.barrage.chunk; -import gnu.trove.iterator.TLongIterator; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.util.QueryConstants.*; @@ -165,7 +165,7 @@ static WritableChunk extractChunkFromInputStream( final int elementSize, final StreamReaderOptions options, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, @@ -179,15 +179,15 @@ static WritableChunk extractChunkFromInputStreamWithConversion( final StreamReaderOptions options, final ShortConversion conversion, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final WritableShortChunk chunk; if (outChunk != null) { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java index c4ef7d82d4c..3b5fb96ab35 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarBinaryChunkInputStreamGenerator.java @@ -4,7 +4,6 @@ package io.deephaven.extensions.barrage.chunk; import com.google.common.io.LittleEndianDataOutputStream; -import gnu.trove.iterator.TLongIterator; import io.deephaven.UncheckedDeephavenException; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.*; @@ -26,6 +25,7 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.Iterator; +import java.util.PrimitiveIterator; public class VarBinaryChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { private static final String DEBUG_NAME = "ObjectChunkInputStream Serialization"; @@ -399,15 +399,15 @@ public int drainTo(final OutputStream outputStream) throws IOException { static WritableObjectChunk extractChunkFromInputStream( final DataInput is, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final Mapper mapper, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long offsetsBuffer = bufferInfoIter.next(); - final long payloadBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long offsetsBuffer = bufferInfoIter.nextLong(); + final long payloadBuffer = bufferInfoIter.nextLong(); final int numElements = nodeInfo.numElements; final WritableObjectChunk chunk; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java index 8b99b1e783f..f9f16d2974b 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VarListChunkInputStreamGenerator.java @@ -4,7 +4,6 @@ package io.deephaven.extensions.barrage.chunk; import com.google.common.io.LittleEndianDataOutputStream; -import gnu.trove.iterator.TLongIterator; import io.deephaven.UncheckedDeephavenException; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; @@ -28,6 +27,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; public class VarListChunkInputStreamGenerator extends BaseChunkInputStreamGenerator> { private static final String DEBUG_NAME = "VarListChunkInputStreamGenerator"; @@ -237,15 +237,15 @@ static WritableObjectChunk extractChunkFromInputStream( final StreamReaderOptions options, final Class type, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long offsetsBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long offsetsBuffer = bufferInfoIter.nextLong(); final Class componentType = type.getComponentType(); final Class innerComponentType = componentType != null ? componentType.getComponentType() : null; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java index 4720adf7ed2..d5b6cbea7da 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/chunk/VectorChunkInputStreamGenerator.java @@ -4,7 +4,6 @@ package io.deephaven.extensions.barrage.chunk; import com.google.common.io.LittleEndianDataOutputStream; -import gnu.trove.iterator.TLongIterator; import io.deephaven.UncheckedDeephavenException; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.ObjectChunk; @@ -29,6 +28,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Iterator; +import java.util.PrimitiveIterator; public class VectorChunkInputStreamGenerator extends BaseChunkInputStreamGenerator, Values>> { private static final String DEBUG_NAME = "VarListChunkInputStreamGenerator"; @@ -238,15 +238,15 @@ static WritableObjectChunk, Values> extractChunkFromInputStream( final Class> type, final Class inComponentType, final Iterator fieldNodeIter, - final TLongIterator bufferInfoIter, + final PrimitiveIterator.OfLong bufferInfoIter, final DataInput is, final WritableChunk outChunk, final int outOffset, final int totalRows) throws IOException { final FieldNodeInfo nodeInfo = fieldNodeIter.next(); - final long validityBuffer = bufferInfoIter.next(); - final long offsetsBuffer = bufferInfoIter.next(); + final long validityBuffer = bufferInfoIter.nextLong(); + final long offsetsBuffer = bufferInfoIter.nextLong(); final Class componentType = VectorExpansionKernel.getComponentType(type, inComponentType); final ChunkType chunkType = ChunkType.fromElementType(componentType); diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java index b655e4fc986..284697fe86d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/ArrowToTableConverter.java @@ -6,8 +6,6 @@ import com.google.common.io.LittleEndianDataInputStream; import com.google.protobuf.CodedInputStream; import com.google.rpc.Code; -import gnu.trove.iterator.TLongIterator; -import gnu.trove.list.array.TLongArrayList; import io.deephaven.UncheckedDeephavenException; import io.deephaven.chunk.ChunkType; import io.deephaven.engine.rowset.RowSetFactory; @@ -29,7 +27,9 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.ArrayList; +import java.util.Arrays; import java.util.Iterator; +import java.util.PrimitiveIterator; import static io.deephaven.extensions.barrage.util.BarrageProtoUtil.DEFAULT_SER_OPTIONS; @@ -167,7 +167,7 @@ protected BarrageMessage createBarrageMessage(BarrageProtoUtil.MessageInfo mi, i new FlatBufferIteratorAdapter<>(batch.nodesLength(), i -> new ChunkInputStreamGenerator.FieldNodeInfo(batch.nodes(i))); - final TLongArrayList bufferInfo = new TLongArrayList(batch.buffersLength()); + final long[] bufferInfo = new long[batch.buffersLength()]; for (int bi = 0; bi < batch.buffersLength(); ++bi) { int offset = LongSizedDataStructure.intSize("BufferInfo", batch.buffers(bi).offset()); int length = LongSizedDataStructure.intSize("BufferInfo", batch.buffers(bi).length()); @@ -178,9 +178,9 @@ protected BarrageMessage createBarrageMessage(BarrageProtoUtil.MessageInfo mi, i // our parsers handle overhanging buffers length += Math.max(0, nextOffset - offset - length); } - bufferInfo.add(length); + bufferInfo[bi] = length; } - final TLongIterator bufferInfoIter = bufferInfo.iterator(); + final PrimitiveIterator.OfLong bufferInfoIter = Arrays.stream(bufferInfo).iterator(); msg.rowsRemoved = RowSetFactory.empty(); msg.shifted = RowSetShiftData.EMPTY; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageChunkAppendingMarshaller.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageChunkAppendingMarshaller.java index 9951f1aef4f..3813525048d 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageChunkAppendingMarshaller.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageChunkAppendingMarshaller.java @@ -2,8 +2,6 @@ import com.google.common.io.LittleEndianDataInputStream; import com.google.protobuf.CodedInputStream; -import gnu.trove.iterator.TLongIterator; -import gnu.trove.list.array.TLongArrayList; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.extensions.barrage.BarrageSnapshotOptions; @@ -22,7 +20,9 @@ import java.io.InputStream; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Iterator; +import java.util.PrimitiveIterator; /** * This class is used to append the results of a DoGet directly into destination {@link WritableChunk}. @@ -140,7 +140,7 @@ public Integer parse(final InputStream stream) { new FlatBufferIteratorAdapter<>(batch.nodesLength(), i -> new ChunkInputStreamGenerator.FieldNodeInfo(batch.nodes(i))); - final TLongArrayList bufferInfo = new TLongArrayList(batch.buffersLength()); + final long[] bufferInfo = new long[batch.buffersLength()]; for (int bi = 0; bi < batch.buffersLength(); ++bi) { int offset = LongSizedDataStructure.intSize("BufferInfo", batch.buffers(bi).offset()); int length = LongSizedDataStructure.intSize("BufferInfo", batch.buffers(bi).length()); @@ -150,9 +150,9 @@ public Integer parse(final InputStream stream) { // our parsers handle overhanging buffers length += Math.max(0, nextOffset - offset - length); } - bufferInfo.add(length); + bufferInfo[bi] = length; } - final TLongIterator bufferInfoIter = bufferInfo.iterator(); + final PrimitiveIterator.OfLong bufferInfoIter = Arrays.stream(bufferInfo).iterator(); for (int ci = 0; ci < destChunks.length; ++ci) { final WritableChunk dest = destChunks[ci]; diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java index 59a76b15b57..94e32aff991 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/util/BarrageStreamReader.java @@ -6,8 +6,6 @@ import com.fasterxml.jackson.databind.util.ByteBufferBackedInputStream; import com.google.common.io.LittleEndianDataInputStream; import com.google.protobuf.CodedInputStream; -import gnu.trove.iterator.TLongIterator; -import gnu.trove.list.array.TLongArrayList; import io.deephaven.barrage.flatbuf.BarrageMessageType; import io.deephaven.barrage.flatbuf.BarrageMessageWrapper; import io.deephaven.barrage.flatbuf.BarrageModColumnMetadata; @@ -33,8 +31,10 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.BitSet; import java.util.Iterator; +import java.util.PrimitiveIterator; import java.util.function.LongConsumer; public class BarrageStreamReader implements StreamReader { @@ -198,7 +198,7 @@ public BarrageMessage safelyParseFrom(final StreamReaderOptions options, new FlatBufferIteratorAdapter<>(batch.nodesLength(), i -> new ChunkInputStreamGenerator.FieldNodeInfo(batch.nodes(i))); - final TLongArrayList bufferInfo = new TLongArrayList(batch.buffersLength()); + final long[] bufferInfo = new long[batch.buffersLength()]; for (int bi = 0; bi < batch.buffersLength(); ++bi) { int offset = LongSizedDataStructure.intSize("BufferInfo", batch.buffers(bi).offset()); int length = LongSizedDataStructure.intSize("BufferInfo", batch.buffers(bi).length()); @@ -208,9 +208,9 @@ public BarrageMessage safelyParseFrom(final StreamReaderOptions options, // our parsers handle overhanging buffers length += Math.max(0, nextOffset - offset - length); } - bufferInfo.add(length); + bufferInfo[bi] = length; } - final TLongIterator bufferInfoIter = bufferInfo.iterator(); + final PrimitiveIterator.OfLong bufferInfoIter = Arrays.stream(bufferInfo).iterator(); // add and mod rows are never combined in a batch. all added rows must be received before the first // mod rows will be received. 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 986d928b52d..01bba46976b 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 @@ -4,7 +4,6 @@ package io.deephaven.extensions.barrage.chunk; import com.google.common.io.LittleEndianDataInputStream; -import gnu.trove.list.array.TLongArrayList; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; @@ -37,9 +36,11 @@ import java.io.IOException; import java.time.Instant; import java.util.ArrayList; +import java.util.Arrays; import java.util.Random; import java.util.function.Consumer; import java.util.function.IntFunction; +import java.util.stream.LongStream; public class BarrageColumnRoundTripTest extends RefreshingTableTestCase { @@ -570,14 +571,15 @@ private static void testRoundTripSerialization( final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); - final TLongArrayList bufferNodes = new TLongArrayList(); + final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); column.drainTo(baos); final DataInput dis = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); try (final WritableChunk rtData = ChunkInputStreamGenerator.extractChunkFromInputStream(options, - chunkType, type, type.getComponentType(), fieldNodes.iterator(), bufferNodes.iterator(), + chunkType, type, type.getComponentType(), 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); @@ -593,14 +595,15 @@ private static void testRoundTripSerialization( final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); - final TLongArrayList bufferNodes = new TLongArrayList(); + final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); column.drainTo(baos); final DataInput dis = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); try (final WritableChunk rtData = ChunkInputStreamGenerator.extractChunkFromInputStream(options, - chunkType, type, type.getComponentType(), fieldNodes.iterator(), bufferNodes.iterator(), + chunkType, type, type.getComponentType(), fieldNodes.iterator(), + bufferNodes.build().iterator(), dis, null, 0, 0)) { Assert.eq(rtData.size(), "rtData.size()", 0); } @@ -623,14 +626,15 @@ private static void testRoundTripSerialization( final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); - final TLongArrayList bufferNodes = new TLongArrayList(); + final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); column.drainTo(baos); final DataInput dis = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); try (final WritableChunk rtData = ChunkInputStreamGenerator.extractChunkFromInputStream(options, - chunkType, type, type.getComponentType(), fieldNodes.iterator(), bufferNodes.iterator(), + chunkType, type, type.getComponentType(), 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); @@ -646,8 +650,9 @@ private static void testRoundTripSerialization( final ArrayList fieldNodes = new ArrayList<>(); column.visitFieldNodes((numElements, nullCount) -> fieldNodes .add(new ChunkInputStreamGenerator.FieldNodeInfo(numElements, nullCount))); - final TLongArrayList bufferNodes = new TLongArrayList(); + final LongStream.Builder bufferNodes = LongStream.builder(); column.visitBuffers(bufferNodes::add); + final long[] buffers = bufferNodes.build().toArray(); column.drainTo(baos); // first message @@ -655,7 +660,8 @@ private static void testRoundTripSerialization( new ByteArrayInputStream(baos.peekBuffer(), 0, baos.size())); try (final WritableChunk rtData = ChunkInputStreamGenerator.extractChunkFromInputStream(options, - chunkType, type, type.getComponentType(), fieldNodes.iterator(), bufferNodes.iterator(), + chunkType, type, type.getComponentType(), fieldNodes.iterator(), + Arrays.stream(buffers).iterator(), dis, null, 0, data.size() * 2)) { // second message dis = new LittleEndianDataInputStream( @@ -663,7 +669,7 @@ private static void testRoundTripSerialization( final WritableChunk rtData2 = ChunkInputStreamGenerator.extractChunkFromInputStream(options, chunkType, type, type.getComponentType(), fieldNodes.iterator(), - bufferNodes.iterator(), + Arrays.stream(buffers).iterator(), dis, rtData, data.size(), data.size() * 2); Assert.eq(rtData, "rtData", rtData2, "rtData2"); validator.assertExpected(data, rtData, null, 0);