From cf2832fb797a95dc8edd66afe03174480e4f7cab Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 27 Dec 2023 00:52:00 -0500 Subject: [PATCH 1/7] Solidify the "contract" re: GetContext and FillContext passed from RegionedPageStores to ColumnRegions. Add an API to ColumnChunkPageStore that will allow different contexts to be used, and integrate that into ParquetColumnRegionBase. Random cleanups in related code. --- .../io/deephaven/engine/table/Context.java | 2 +- .../engine/table/impl/ContextWithChunk.java | 6 +- .../io/deephaven/engine/page/PageStore.java | 15 ++-- .../regioned/DeferredColumnRegionBase.java | 10 --- .../regioned/GenericColumnRegionBase.java | 19 +++++ .../sources/regioned/RegionContextHolder.java | 80 +++++++++++++++---- .../sources/regioned/RegionedPageStore.java | 5 +- .../table/pagestore/ColumnChunkPageStore.java | 32 +++++--- .../OffsetIndexBasedColumnChunkPageStore.java | 18 +++-- .../VariablePageSizeColumnChunkPageStore.java | 18 +++-- .../table/region/ParquetColumnRegionBase.java | 67 +++++++++++----- .../AppendOnlyFixedSizePageRegionByte.java | 6 +- .../AppendOnlyFixedSizePageRegionChar.java | 6 +- .../AppendOnlyFixedSizePageRegionDouble.java | 6 +- .../AppendOnlyFixedSizePageRegionFloat.java | 6 +- .../AppendOnlyFixedSizePageRegionInt.java | 6 +- .../AppendOnlyFixedSizePageRegionLong.java | 6 +- .../AppendOnlyFixedSizePageRegionObject.java | 6 +- .../AppendOnlyFixedSizePageRegionShort.java | 6 +- 19 files changed, 221 insertions(+), 99 deletions(-) diff --git a/engine/api/src/main/java/io/deephaven/engine/table/Context.java b/engine/api/src/main/java/io/deephaven/engine/table/Context.java index ed6e28cd4ab..2774e5d4d43 100644 --- a/engine/api/src/main/java/io/deephaven/engine/table/Context.java +++ b/engine/api/src/main/java/io/deephaven/engine/table/Context.java @@ -10,9 +10,9 @@ * Source, Functor or Sink. */ public interface Context extends SafeCloseable { + /** * Release any resources associated with this context. The context should not be used afterwards. */ default void close() {} - } diff --git a/engine/base/src/main/java/io/deephaven/engine/table/impl/ContextWithChunk.java b/engine/base/src/main/java/io/deephaven/engine/table/impl/ContextWithChunk.java index e9d4f00f7f3..f76ba38cfa8 100644 --- a/engine/base/src/main/java/io/deephaven/engine/table/impl/ContextWithChunk.java +++ b/engine/base/src/main/java/io/deephaven/engine/table/impl/ContextWithChunk.java @@ -63,14 +63,14 @@ public CONTEXT getContext() { * @return The context held in this Context */ public static CONTEXT getContext(@NotNull Context context) { - // noinspection unchecked + // noinspection unchecked,rawtypes return (CONTEXT) ((ContextWithChunk) context).context; } /** - * Makes sure that the internal array (and hence the writableChunk) is at least specified size. + * Makes sure that the internal array (and hence the writableChunk) is at least the specified size. */ - public void ensureLength(final int length) { + public void ensureSize(final int length) { if (writableChunk.size() < length) { if (writableChunk.capacity() < length) { final SafeCloseable oldWritableChunk = writableChunk; diff --git a/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java b/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java index a49dd1fd514..76ebe61acf1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequenceFactory; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * PageStores are a collection of non-overlapping pages, which provides a single {@link ChunkSource} interface across @@ -21,14 +22,16 @@ public interface PageStore, DefaultChunkSource.SupportsContiguousGet { /** - * @return The page containing row, after applying {@link #mask()}. + * @param fillContext The fill context to use; may be {@code null} if the calling code does not have a fill context + * @param rowKey The row key to get the page for + * @return The page containing {@code rowKey}, after applying {@link #mask()}. */ @NotNull - PAGE getPageContaining(FillContext fillContext, long row); + PAGE getPageContaining(@Nullable FillContext fillContext, long rowKey); @Override default Chunk getChunk(@NotNull final GetContext context, @NotNull final RowSequence rowSequence) { - if (rowSequence.size() == 0) { + if (rowSequence.isEmpty()) { return getChunkType().getEmptyChunk(); } @@ -65,9 +68,11 @@ default Chunk getChunk(@NotNull final GetContext context, final } @Override - default void fillChunk(@NotNull final FillContext context, @NotNull final WritableChunk destination, + default void fillChunk( + @NotNull final FillContext context, + @NotNull final WritableChunk destination, @NotNull final RowSequence rowSequence) { - if (rowSequence.size() == 0) { + if (rowSequence.isEmpty()) { return; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/DeferredColumnRegionBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/DeferredColumnRegionBase.java index 37258661c0e..b0e4bc0f91b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/DeferredColumnRegionBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/DeferredColumnRegionBase.java @@ -99,14 +99,4 @@ public Chunk getChunk(@NotNull GetContext context, @NotNull RowS public Chunk getChunk(@NotNull GetContext context, long firstKey, long lastKey) { return getResultRegion().getChunk(context, firstKey, lastKey); } - - @Override - public FillContext makeFillContext(int chunkCapacity, SharedContext sharedContext) { - return getResultRegion().makeFillContext(chunkCapacity, sharedContext); - } - - @Override - public GetContext makeGetContext(int chunkCapacity, SharedContext sharedContext) { - return getResultRegion().makeGetContext(chunkCapacity, sharedContext); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java index 5f24d1737bb..93185889d28 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java @@ -4,7 +4,9 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.chunk.attributes.Any; +import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; +import org.jetbrains.annotations.Nullable; /** * Base {@link ColumnRegion} implementation. @@ -33,4 +35,21 @@ protected final void throwIfInvalidated() { throw new InvalidatedRegionException("Column region has been invalidated due to data removal"); } } + + @Override + public final FillContext makeFillContext(final int chunkCapacity, @Nullable final SharedContext sharedContext) { + throw new UnsupportedOperationException( + "ColumnRegions do not support makeFillContext. Implementations should accept " + + "io.deephaven.engine.table.impl.sources.regioned.RegionContextHolder and get/set/update " + + "their own context as needed."); + } + + @Override + public final GetContext makeGetContext(final int chunkCapacity, @Nullable final SharedContext sharedContext) { + throw new UnsupportedOperationException( + "ColumnRegions do not support makeGetContext. Implementations should accept " + + "io.deephaven.engine.table.impl.DefaultGetContext with a " + + "io.deephaven.engine.table.impl.sources.regioned.RegionContextHolder as FillContext, and " + + "get/set/update their own context as needed."); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java index 50502bd1676..829d14099e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java @@ -6,31 +6,38 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.Context; import io.deephaven.engine.table.SharedContext; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +/** + * {@link ChunkSource.FillContext} implementation for use by {@link RegionedPageStore} implementations. This is + * basically a re-usable box around an inner {@link Context context} object, filled with whatever the most recently used + * region chose to store. + */ public class RegionContextHolder implements ChunkSource.FillContext { + private final int chunkCapacity; private final SharedContext sharedContext; + private Context innerContext; public RegionContextHolder(final int chunkCapacity, @Nullable final SharedContext sharedContext) { - this.chunkCapacity = chunkCapacity; this.sharedContext = sharedContext; } - @Override - public boolean supportsUnboundedFill() { - return true; - } - /** - * Set the inner wrapped context object for use by downstream regions. + * {@inheritDoc} * - * @param contextObject The context object + * @implNote This implementation always returns {@code true}, as the {@link RegionedPageStore} fill implementation + * follows an append pattern over multiple regions when necessary, and all known inner context + * implementations trivially support unbounded fill. We thus make this a requirement for future inner + * context implementations, either naturally or via a slicing/looping pattern. */ - public void setInnerContext(@Nullable final Context contextObject) { - this.innerContext = contextObject; + @Override + public boolean supportsUnboundedFill() { + return true; } /** @@ -54,7 +61,7 @@ public SharedContext getSharedContext() { /** * Get the inner context value set by {@link #setInnerContext(Context)} and cast it to the templated type. * - * @return The inner context value. + * @return The inner context value * @param The desired result type */ public T getInnerContext() { @@ -62,11 +69,54 @@ public T getInnerContext() { return (T) innerContext; } + /** + * Set the inner context object for use by the current region. The previous inner context will be + * {@link SafeCloseable#close() closed}. + * + * @param newInnerContext The new context object + */ + public void setInnerContext(@Nullable final Context newInnerContext) { + if (newInnerContext == innerContext) { + return; + } + try (final SafeCloseable ignoredOldInnerContext = innerContext) { + innerContext = newInnerContext; + } + } + + @FunctionalInterface + public interface Updater { + /** + * Provide a new inner context value based on the current state of this holder. + * + * @param chunkCapacity The holder's {@link #getChunkCapacity() chunk capacity} + * @param sharedContext The holder's {@link #getSharedContext() SharedContext} + * @param currentInnerContext The holder's {@link #getInnerContext() current inner context} + * @return The new inner context to be held by this holder + * @param The result type + */ + @Nullable + T updateInnerContext( + int chunkCapacity, + @Nullable final SharedContext sharedContext, + @Nullable final Context currentInnerContext); + } + + /** + * Update the inner context value using the provided updater. + * + * @param updater The {@link Updater} to use + * @return The result of {@code updater} + * @param The desired result type + */ + public T updateInnerContext(@NotNull final Updater updater) { + final T newInnerContext = updater.updateInnerContext(chunkCapacity, sharedContext, innerContext); + setInnerContext(newInnerContext); + return newInnerContext; + } + @Override public void close() { - if (innerContext != null) { - innerContext.close(); - innerContext = null; - } + setInnerContext(null); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java index b429158ba07..5f28b010cb7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java @@ -12,6 +12,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; public interface RegionedPageStore> extends PageStore { @@ -86,8 +87,8 @@ default REGION_TYPE lookupRegion(final long elementRowKey) { @Override @NotNull @FinalDefault - default REGION_TYPE getPageContaining(final FillContext fillContext, final long row) { - return lookupRegion(row); + default REGION_TYPE getPageContaining(@Nullable final FillContext fillContext, final long rowKey) { + return lookupRegion(rowKey); } @Override diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java index dcd7677ccf5..4f93849f460 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java @@ -5,6 +5,8 @@ import io.deephaven.base.verify.Require; import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.Context; +import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.engine.table.Releasable; import io.deephaven.chunk.attributes.Any; @@ -19,6 +21,7 @@ import io.deephaven.util.SafeCloseable; import io.deephaven.vector.Vector; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.VisibleForTesting; import java.io.IOException; @@ -42,7 +45,8 @@ public static class CreatorResult { public final Supplier> dictionaryChunkSupplier; public final ColumnChunkPageStore dictionaryKeysPageStore; - private CreatorResult(@NotNull final ColumnChunkPageStore pageStore, + private CreatorResult( + @NotNull final ColumnChunkPageStore pageStore, final Supplier> dictionaryChunkSupplier, final ColumnChunkPageStore dictionaryKeysPageStore) { this.pageStore = pageStore; @@ -51,7 +55,8 @@ private CreatorResult(@NotNull final ColumnChunkPageStore pageStore, } } - private static boolean canUseOffsetIndexBasedPageStore(@NotNull final ColumnChunkReader columnChunkReader, + private static boolean canUseOffsetIndexBasedPageStore( + @NotNull final ColumnChunkReader columnChunkReader, @NotNull final ColumnDefinition columnDefinition) { if (columnChunkReader.getOffsetIndex() == null) { return false; @@ -112,7 +117,8 @@ public static CreatorResult create( dictionaryKeysColumnChunkPageStore); } - ColumnChunkPageStore(@NotNull final PageCache pageCache, + ColumnChunkPageStore( + @NotNull final PageCache pageCache, @NotNull final ColumnChunkReader columnChunkReader, final long mask, final ToPage toPage) throws IOException { @@ -154,14 +160,6 @@ public ChunkType getChunkType() { return toPage.getChunkType(); } - /** - * These implementations don't use the FillContext parameter, so we're create a helper method to ignore it. - */ - @NotNull - public ChunkPage getPageContaining(final long row) { - return getPageContaining(DEFAULT_FILL_INSTANCE, row); - } - /** * @see ColumnChunkReader#usesDictionaryOnEveryPage() */ @@ -171,4 +169,16 @@ public boolean usesDictionaryOnEveryPage() { @Override public void close() {} + + public boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { + return currentInnerContext == DEFAULT_FILL_INSTANCE; + } + + public boolean isGetContextCompatible(@Nullable final Context currentInnerContext) { + if (!(currentInnerContext instanceof DefaultGetContext)) { + return false; + } + final DefaultGetContext getContext = (DefaultGetContext) currentInnerContext; + return isFillContextCompatible(getContext.getFillContext()); + } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java index 62f0f22cfa0..71e9ffbe4fa 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java @@ -12,6 +12,7 @@ import io.deephaven.parquet.base.ColumnPageReader; import org.apache.parquet.internal.column.columnindex.OffsetIndex; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.io.UncheckedIOException; @@ -47,7 +48,8 @@ private static final class PageState { private final AtomicReferenceArray> pageStates; private final ColumnChunkReader.ColumnPageDirectAccessor columnPageDirectAccessor; - OffsetIndexBasedColumnChunkPageStore(@NotNull final PageCache pageCache, + OffsetIndexBasedColumnChunkPageStore( + @NotNull final PageCache pageCache, @NotNull final ColumnChunkReader columnChunkReader, final long mask, @NotNull final ToPage toPage) throws IOException { @@ -127,21 +129,21 @@ private ChunkPage getPage(final int pageNum) { return page.getPage(); } - @NotNull @Override - public ChunkPage getPageContaining(@NotNull final FillContext fillContext, long row) { - row &= mask(); - Require.inRange(row, "row", numRows(), "numRows"); + @NotNull + public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { + rowKey &= mask(); + Require.inRange(rowKey, "row", numRows(), "numRows"); int pageNum; if (fixedPageSize == PAGE_SIZE_NOT_FIXED) { - pageNum = findPageNumUsingOffsetIndex(offsetIndex, row); + pageNum = findPageNumUsingOffsetIndex(offsetIndex, rowKey); } else { - pageNum = (int) (row / fixedPageSize); + pageNum = (int) (rowKey / fixedPageSize); if (pageNum >= numPages) { // This can happen if the last page is larger than rest of the pages, which are all the same size. // We have already checked that row is less than numRows. - Assert.geq(row, "row", offsetIndex.getFirstRowIndex(numPages - 1), + Assert.geq(rowKey, "row", offsetIndex.getFirstRowIndex(numPages - 1), "offsetIndex.getFirstRowIndex(numPages - 1)"); pageNum = (numPages - 1); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java index 9975ebdbb5d..ee8239c38cb 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java @@ -11,6 +11,7 @@ import io.deephaven.parquet.base.ColumnChunkReader; import io.deephaven.parquet.base.ColumnPageReader; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.io.IOException; import java.io.UncheckedIOException; @@ -31,7 +32,8 @@ final class VariablePageSizeColumnChunkPageStore extends Colum private final Iterator columnPageReaderIterator; private volatile WeakReference>[] pages; - VariablePageSizeColumnChunkPageStore(@NotNull final PageCache pageCache, + VariablePageSizeColumnChunkPageStore( + @NotNull final PageCache pageCache, @NotNull final ColumnChunkReader columnChunkReader, final long mask, @NotNull final ToPage toPage) throws IOException { @@ -136,23 +138,23 @@ private ChunkPage getPage(final int pageNum) { return page.getPage(); } - @NotNull @Override - public ChunkPage getPageContaining(@NotNull final FillContext fillContext, long row) { - row &= mask(); - Require.inRange(row - pageRowOffsets[0], "row", numRows(), "numRows"); + @NotNull + public ChunkPage getPageContaining(@Nullable final FillContext fillContext, long rowKey) { + rowKey &= mask(); + Require.inRange(rowKey - pageRowOffsets[0], "row", numRows(), "numRows"); int localNumPages = numPages; - int pageNum = Arrays.binarySearch(pageRowOffsets, 1, localNumPages + 1, row); + int pageNum = Arrays.binarySearch(pageRowOffsets, 1, localNumPages + 1, rowKey); if (pageNum < 0) { pageNum = -2 - pageNum; } if (pageNum >= localNumPages) { - int minPageNum = fillToRow(localNumPages, row); + int minPageNum = fillToRow(localNumPages, rowKey); localNumPages = numPages; - pageNum = Arrays.binarySearch(pageRowOffsets, minPageNum + 1, localNumPages + 1, row); + pageNum = Arrays.binarySearch(pageRowOffsets, minPageNum + 1, localNumPages + 1, rowKey); if (pageNum < 0) { pageNum = -2 - pageNum; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java index 98377e09627..b7dee8e71c2 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java @@ -4,15 +4,19 @@ package io.deephaven.parquet.table.region; import io.deephaven.base.verify.Require; +import io.deephaven.engine.table.Context; +import io.deephaven.engine.table.SharedContext; +import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; +import io.deephaven.engine.table.impl.sources.regioned.RegionContextHolder; import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.Chunk; -import io.deephaven.engine.table.SharedContext; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.ChunkPage; import io.deephaven.engine.rowset.RowSequence; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import javax.annotation.OverridingMethodsMustInvokeSuper; @@ -28,29 +32,34 @@ public abstract class ParquetColumnRegionBase // We are making the following assumptions, so these basic functions are inlined rather than virtual calls. Require.eq(columnChunkPageStore.mask(), "columnChunkPageStore.mask()", mask(), "ColumnRegion.mask()"); - Require.eq(columnChunkPageStore.firstRowOffset(), "columnChunkPageStore.firstRowOffset()", firstRowOffset(), - "ColumnRegion.firstrRowOffset()"); + Require.eq(columnChunkPageStore.firstRowOffset(), "columnChunkPageStore.firstRowOffset()", + firstRowOffset(), "ColumnRegion.firstRowOffset()"); } @Override - public final Chunk getChunk(@NotNull final GetContext context, + public final Chunk getChunk( + @NotNull final GetContext context, @NotNull final RowSequence rowSequence) { throwIfInvalidated(); - return columnChunkPageStore.getChunk(context, rowSequence); + return columnChunkPageStore.getChunk(innerGetContext(context), rowSequence); } @Override - public final Chunk getChunk(@NotNull final GetContext context, final long firstKey, + public final Chunk getChunk( + @NotNull final GetContext context, + final long firstKey, final long lastKey) { throwIfInvalidated(); - return columnChunkPageStore.getChunk(context, firstKey, lastKey); + return columnChunkPageStore.getChunk(innerGetContext(context), firstKey, lastKey); } @Override - public final void fillChunk(@NotNull final FillContext context, - @NotNull final WritableChunk destination, @NotNull final RowSequence rowSequence) { + public final void fillChunk( + @NotNull final FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { throwIfInvalidated(); - columnChunkPageStore.fillChunk(context, destination, rowSequence); + columnChunkPageStore.fillChunk(innerFillContext(context), destination, rowSequence); } @Override @@ -59,13 +68,13 @@ public final void fillChunkAppend( @NotNull final WritableChunk destination, @NotNull final RowSequence.Iterator rowSequenceIterator) { throwIfInvalidated(); - columnChunkPageStore.fillChunkAppend(context, destination, rowSequenceIterator); + columnChunkPageStore.fillChunkAppend(innerFillContext(context), destination, rowSequenceIterator); } @Override public final ChunkPage getChunkPageContaining(final long elementIndex) { throwIfInvalidated(); - return columnChunkPageStore.getPageContaining(elementIndex); + return columnChunkPageStore.getPageContaining(null, elementIndex); } @Override @@ -75,15 +84,33 @@ public void releaseCachedResources() { columnChunkPageStore.releaseCachedResources(); } - @Override - public final FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { - throwIfInvalidated(); - return columnChunkPageStore.makeFillContext(chunkCapacity, sharedContext); + private FillContext innerFillContext(@NotNull final FillContext context) { + return ((RegionContextHolder) context) + .updateInnerContext(this::fillContextUpdater); } - @Override - public final GetContext makeGetContext(final int chunkCapacity, final SharedContext sharedContext) { - throwIfInvalidated(); - return columnChunkPageStore.makeGetContext(chunkCapacity, sharedContext); + private T fillContextUpdater( + int chunkCapacity, + @Nullable final SharedContext sharedContext, + @Nullable final Context currentInnerContext) { + // noinspection unchecked + return (T) (columnChunkPageStore.isFillContextCompatible(currentInnerContext) + ? currentInnerContext + : columnChunkPageStore.makeFillContext(chunkCapacity, sharedContext)); + } + + private GetContext innerGetContext(@NotNull final GetContext context) { + return ((RegionContextHolder) DefaultGetContext.getFillContext(context)) + .updateInnerContext(this::getContextUpdater); + } + + private T getContextUpdater( + int chunkCapacity, + @Nullable final SharedContext sharedContext, + @Nullable final Context currentInnerContext) { + // noinspection unchecked + return (T) (columnChunkPageStore.isGetContextCompatible(currentInnerContext) + ? currentInnerContext + : columnChunkPageStore.makeGetContext(chunkCapacity, sharedContext)); } } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionByte.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionByte.java index c1759a33384..4bf4236e132 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionByte.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionByte.java @@ -11,7 +11,6 @@ import io.deephaven.engine.rowset.RowSequenceFactory; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -20,6 +19,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageByte; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -75,7 +75,9 @@ public byte[] getBytes( @Override @NotNull - public final ChunkHolderPageByte getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageByte getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionChar.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionChar.java index 2da72d2f94a..8c415e1d0e4 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionChar.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionChar.java @@ -1,7 +1,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -10,6 +9,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageChar; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -53,7 +53,9 @@ public char getChar(final long rowKey) { @Override @NotNull - public final ChunkHolderPageChar getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageChar getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionDouble.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionDouble.java index a7b3df45eee..c58ebd2f01f 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionDouble.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionDouble.java @@ -6,7 +6,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -15,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageDouble; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -58,7 +58,9 @@ public double getDouble(final long rowKey) { @Override @NotNull - public final ChunkHolderPageDouble getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageDouble getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionFloat.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionFloat.java index b97b9bfc4f5..8556260ef45 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionFloat.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionFloat.java @@ -6,7 +6,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -15,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageFloat; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -58,7 +58,9 @@ public float getFloat(final long rowKey) { @Override @NotNull - public final ChunkHolderPageFloat getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageFloat getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionInt.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionInt.java index 72c581a0846..3a510bdfb73 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionInt.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionInt.java @@ -6,7 +6,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -15,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageInt; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -58,7 +58,9 @@ public int getInt(final long rowKey) { @Override @NotNull - public final ChunkHolderPageInt getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageInt getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionLong.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionLong.java index 9b9bbc4f294..39e8aaba8d3 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionLong.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionLong.java @@ -6,7 +6,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -15,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageLong; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -58,7 +58,9 @@ public long getLong(final long rowKey) { @Override @NotNull - public final ChunkHolderPageLong getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageLong getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionObject.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionObject.java index 6e57df6c67c..8c3526fcc6a 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionObject.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionObject.java @@ -6,7 +6,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -15,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageObject; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -58,7 +58,9 @@ public T getObject(final long rowKey) { @Override @NotNull - public final ChunkHolderPageObject getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageObject getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionShort.java b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionShort.java index d6081a5b9c1..99d2d605c42 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionShort.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/region/AppendOnlyFixedSizePageRegionShort.java @@ -6,7 +6,6 @@ package io.deephaven.generic.region; import io.deephaven.base.MathUtil; -import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PageStore; @@ -15,6 +14,7 @@ import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; import io.deephaven.generic.page.ChunkHolderPageShort; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import java.lang.ref.SoftReference; import java.util.Arrays; @@ -58,7 +58,9 @@ public short getShort(final long rowKey) { @Override @NotNull - public final ChunkHolderPageShort getPageContaining(final FillContext fillContext, final long rowKey) { + public final ChunkHolderPageShort getPageContaining( + @Nullable final FillContext fillContext, + final long rowKey) { return getPageContaining(rowKey); } From ba00a55bed07119428e91811aeb2c8bfd45bc84f Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 27 Dec 2023 16:13:41 -0500 Subject: [PATCH 2/7] Fix JavaDoc issues in DateTimeUtils --- .../java/io/deephaven/time/DateTimeUtils.java | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java index 231547ee56f..56a49c8c2ef 100644 --- a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java +++ b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java @@ -21,7 +21,6 @@ import java.time.zone.ZoneRulesException; import java.util.Date; import java.util.Objects; -import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -2904,8 +2903,9 @@ public static Instant lowerBin(@Nullable final Instant instant, long intervalNan * start of the five-minute window that contains the input zoned date time. * * @param dateTime zoned date time for which to evaluate the start of the containing window - * @param intervalNanos size of the window in nanoseconds * @param offset The window start offset in nanoseconds. - * For example, a value of MINUTE would offset all windows by one minute. + * @param intervalNanos size of the window in nanoseconds + * @param offset The window start offset in nanoseconds. For example, a value of MINUTE would offset all windows by + * one minute. * @return {@code null} if either input is {@code null}; otherwise, a {@link ZonedDateTime} representing the start * of the window */ @@ -2926,8 +2926,9 @@ public static ZonedDateTime lowerBin(@Nullable final ZonedDateTime dateTime, lon * five-minute window that contains the input instant. * * @param instant instant for which to evaluate the start of the containing window - * @param intervalNanos size of the window in nanoseconds * @return {@code null} if either input is {@code null}; - * otherwise, an {@link Instant} representing the end of the window + * @param intervalNanos size of the window in nanoseconds + * @return {@code null} if either input is {@code null}; otherwise, an {@link Instant} representing the end of the + * window */ @ScriptApi @Nullable @@ -2945,8 +2946,9 @@ public static Instant upperBin(@Nullable final Instant instant, long intervalNan * the five-minute window that contains the input zoned date time. * * @param dateTime zoned date time for which to evaluate the start of the containing window - * @param intervalNanos size of the window in nanoseconds * @return {@code null} if either input is {@code null}; - * otherwise, a {@link ZonedDateTime} representing the end of the window + * @param intervalNanos size of the window in nanoseconds + * @return {@code null} if either input is {@code null}; otherwise, a {@link ZonedDateTime} representing the end of + * the window */ @ScriptApi @Nullable @@ -2964,8 +2966,9 @@ public static ZonedDateTime upperBin(@Nullable final ZonedDateTime dateTime, lon * five-minute window that contains the input instant. * * @param instant instant for which to evaluate the start of the containing window - * @param intervalNanos size of the window in nanoseconds * @param offset The window start offset in nanoseconds. - * For example, a value of MINUTE would offset all windows by one minute. + * @param intervalNanos size of the window in nanoseconds + * @param offset The window start offset in nanoseconds. For example, a value of MINUTE would offset all windows by + * one minute. * @return {@code null} if either input is {@code null}; otherwise, an {@link Instant} representing the end of the * window */ @@ -2986,7 +2989,8 @@ public static Instant upperBin(@Nullable final Instant instant, long intervalNan * the five-minute window that contains the input zoned date time. * * @param dateTime zoned date time for which to evaluate the start of the containing window - * @param intervalNanos size of the window in nanoseconds * @param offset The window start offset in nanoseconds. + * @param intervalNanos size of the window in nanoseconds + * @param offset The window start offset in nanoseconds. * For example, a value of MINUTE would offset all windows by one minute. * @return {@code null} if either input is {@code null}; otherwise, a {@link ZonedDateTime} representing the end of * the window From c92547eef921798e5e2fb75a31046cb5f47468d6 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 29 Dec 2023 14:25:34 -0500 Subject: [PATCH 3/7] Random formatting, JavaDoc, inheritance, and variable name changes --- .../engine/table/impl/DefaultChunkSource.java | 39 +++++++---- .../io/deephaven/engine/page/ChunkPage.java | 4 +- .../java/io/deephaven/engine/page/Page.java | 18 ++--- .../engine/page/PagingChunkSource.java | 70 +++++++++---------- .../region/ParquetColumnRegionObject.java | 3 +- .../generic/page/ChunkHolderPageBoolean.java | 12 ++-- .../generic/page/ChunkHolderPageByte.java | 12 ++-- .../generic/page/ChunkHolderPageChar.java | 12 ++-- .../generic/page/ChunkHolderPageDouble.java | 12 ++-- .../generic/page/ChunkHolderPageFloat.java | 12 ++-- .../generic/page/ChunkHolderPageInt.java | 12 ++-- .../generic/page/ChunkHolderPageLong.java | 12 ++-- .../generic/page/ChunkHolderPageObject.java | 12 ++-- .../generic/page/ChunkHolderPageShort.java | 12 ++-- 14 files changed, 126 insertions(+), 116 deletions(-) diff --git a/engine/base/src/main/java/io/deephaven/engine/table/impl/DefaultChunkSource.java b/engine/base/src/main/java/io/deephaven/engine/table/impl/DefaultChunkSource.java index fbfdff00865..0c586ab25fb 100644 --- a/engine/base/src/main/java/io/deephaven/engine/table/impl/DefaultChunkSource.java +++ b/engine/base/src/main/java/io/deephaven/engine/table/impl/DefaultChunkSource.java @@ -32,7 +32,7 @@ default Chunk getChunk(@NotNull final GetContext context, @NotNu } @Override - default Chunk getChunk(@NotNull final GetContext context, long firstKey, long lastKey) { + default Chunk getChunk(@NotNull final GetContext context, final long firstKey, final long lastKey) { try (RowSequence rowSequence = RowSequenceFactory.forRange(firstKey, lastKey)) { return getChunk(context, rowSequence); } @@ -48,20 +48,25 @@ default Chunk getChunkByFilling(@NotNull final GetContext context, @NotNul interface WithPrev extends DefaultChunkSource, ChunkSource.WithPrev { @Override - default Chunk getPrevChunk(@NotNull final GetContext context, + default Chunk getPrevChunk( + @NotNull final GetContext context, @NotNull final RowSequence rowSequence) { return getPrevChunkByFilling(context, rowSequence); } @Override - default Chunk getPrevChunk(@NotNull final GetContext context, long firstKey, long lastKey) { + default Chunk getPrevChunk( + @NotNull final GetContext context, + final long firstKey, + final long lastKey) { try (RowSequence rowSequence = RowSequenceFactory.forRange(firstKey, lastKey)) { return getPrevChunk(context, rowSequence); } } @FinalDefault - default Chunk getPrevChunkByFilling(@NotNull final GetContext context, + default Chunk getPrevChunkByFilling( + @NotNull final GetContext context, @NotNull final RowSequence rowSequence) { WritableChunk chunk = DefaultGetContext.getWritableChunk(context); fillPrevChunk(DefaultGetContext.getFillContext(context), chunk, rowSequence); @@ -72,35 +77,42 @@ default Chunk getPrevChunkByFilling(@NotNull final GetContext context, default ChunkSource getPrevSource() { final ChunkSource.WithPrev chunkSource = this; - return new ChunkSource() { + return new ChunkSource<>() { @Override public ChunkType getChunkType() { return chunkSource.getChunkType(); } @Override - public Chunk getChunk(@NotNull GetContext context, @NotNull RowSequence rowSequence) { + public Chunk getChunk( + @NotNull final GetContext context, + @NotNull final RowSequence rowSequence) { return chunkSource.getPrevChunk(context, rowSequence); } @Override - public Chunk getChunk(@NotNull GetContext context, long firstKey, long lastKey) { + public Chunk getChunk( + @NotNull final GetContext context, + final long firstKey, + final long lastKey) { return chunkSource.getPrevChunk(context, firstKey, lastKey); } @Override - public void fillChunk(@NotNull FillContext context, @NotNull WritableChunk destination, - @NotNull RowSequence rowSequence) { + public void fillChunk( + @NotNull final FillContext context, + @NotNull final WritableChunk destination, + @NotNull final RowSequence rowSequence) { chunkSource.fillPrevChunk(context, destination, rowSequence); } @Override - public GetContext makeGetContext(int chunkCapacity, SharedContext sharedContext) { + public GetContext makeGetContext(final int chunkCapacity, final SharedContext sharedContext) { return chunkSource.makeGetContext(chunkCapacity, sharedContext); } @Override - public FillContext makeFillContext(int chunkCapacity, SharedContext sharedContext) { + public FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { return chunkSource.makeFillContext(chunkCapacity, sharedContext); } }; @@ -113,7 +125,8 @@ public FillContext makeFillContext(int chunkCapacity, SharedContext sharedContex */ interface SupportsContiguousGet extends DefaultChunkSource { @Override - default Chunk getChunk(@NotNull final GetContext context, + default Chunk getChunk( + @NotNull final GetContext context, @NotNull final RowSequence rowSequence) { return rowSequence.isContiguous() ? getChunk(context, rowSequence.firstRowKey(), rowSequence.lastRowKey()) @@ -121,6 +134,6 @@ default Chunk getChunk(@NotNull final GetContext context, } @Override - Chunk getChunk(@NotNull final GetContext context, long firstKey, long lastKey); + Chunk getChunk(@NotNull GetContext context, long firstKey, long lastKey); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/page/ChunkPage.java b/engine/table/src/main/java/io/deephaven/engine/page/ChunkPage.java index 593a4334ea0..d9193aa83dd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/ChunkPage.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/ChunkPage.java @@ -26,8 +26,8 @@ default long lastRow(final long row) { @Override @FinalDefault - default long maxRow(final long row) { - return lastRow(row); + default long maxRow(final long rowKey) { + return lastRow(rowKey); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/page/Page.java b/engine/table/src/main/java/io/deephaven/engine/page/Page.java index 09fadcda649..3078a69ccbe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/Page.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/Page.java @@ -20,9 +20,9 @@ * Non overlapping pages can be collected together in a {@link PageStore}, which provides the {@link ChunkSource} * interface to the collection of all of its Pages. *

- * There are two distinct use cases/types of pages. The first use case are {@code Page}s which always have a length() + * There are two distinct use cases/types of pages. The first use case are Pages which always have a length() * > 0. These store length() values, which can be assessed via the {@link ChunkSource} methods. Valid - * {@link RowSequence} passed to those methods will have their offset in the range [firstRowOffset(), firstRowOffset() + + * {@link RowSequence row sequences} passed to those methods will have their offset in the range [firstRowOffset(), firstRowOffset() + * length()). Passing OrderKeys with offsets outside of this range will have undefined results. *

* The second use case will always have length() == 0 and firstRowOffset() == 0. These represent "Null" regions which @@ -42,20 +42,20 @@ public interface Page extends PagingChunkSource { long firstRowOffset(); /** - * @param row Any row contained on this page. - * @return the first row of this page, located in the same way as row. + * @param rowKey Any row key contained on this page + * @return The first row key of this page, located in the same way as {@code rowKey} */ @FinalDefault - default long firstRow(final long row) { - return (row & ~mask()) | firstRowOffset(); + default long firstRow(final long rowKey) { + return (rowKey & ~mask()) | firstRowOffset(); } /** - * @return the offset for the given row in this page, in [0, {@code maxRow(row)}]. + * @return The offset for the given row key in this page, in [0, {@code maxRow(rowKey)}]. */ @FinalDefault - default long getRowOffset(long row) { - return (row & mask()) - firstRowOffset(); + default long getRowOffset(final long rowKey) { + return (rowKey & mask()) - firstRowOffset(); } /** diff --git a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java index 652601fdbc8..5708c96c690 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java @@ -10,41 +10,40 @@ import org.jetbrains.annotations.NotNull; /** - * In order to be able to cache and reuse ChunkSources across multiple Tables (or other references), - * {@code PagingChunkSource} adds a mask to the ChunkSource, and supports some additional {@code fillChunk} methods. - * - * The mask is a bitmask of the lower order bits of the keys in an OrderKeys, which specifies the bits from the - * {@link RowSequence} which will be used to uniquely specify the offsets into the ChunkSource elements on calls to - * {@link ChunkSource#fillChunk(FillContext, WritableChunk, RowSequence)}, - * {@link ChunkSource#getChunk(GetContext, RowSequence)}, {@link ChunkSource#getChunk(GetContext, long, long)}. - * - * Also, a new method {@link PagingChunkSource#fillChunkAppend(FillContext, WritableChunk, RowSequence.Iterator)} is - * added, which supports doing a fillChunk incrementally across a series of pages. + * In order to be able to cache and reuse {@link ChunkSource ChunkSources} across multiple tables (or other references), + * {@code PagingChunkSource} adds a {@link #mask()} to {@code ChunkSource} and supports some additional + * {@link #fillChunk} methods. + *

+ * The mask is a bitmask of the lower order bits of the row keys in a {@link RowSequence}, which specifies the bits from + * the {@link RowSequence} which will be used to uniquely specify the offsets into the ChunkSource elements on calls to + * {@link #fillChunk} and {@link #getChunk}. + *

+ * Also, a new method {@link #fillChunkAppend(FillContext, WritableChunk, RowSequence.Iterator)} is added, which + * supports filling a chunk incrementally across a series of pages. */ public interface PagingChunkSource extends ChunkSource { /** - * This mask is applied to {@link RowSequence} which are passed into - * {@link #getChunk(ChunkSource.GetContext, RowSequence)} and - * {@link #fillChunk(ChunkSource.FillContext, WritableChunk, RowSequence)}. This allows the {@link PagingChunkSource - * PagingChunkSources} to be cached, and reused even if they are properly relocated in key space. + * This mask is applied to {@link RowSequence RowSequences} which are passed into {@link #getChunk}, + * {@link #fillChunk}, and {@link #fillChunkAppend(FillContext, WritableChunk, RowSequence.Iterator)} . This allows + * {@code PagingChunkSources} to be cached and reused even if they are properly relocated in key space. * - * @return the mask for this page, which must be a bitmask representing the some number of lower order bits of a - * long. + * @return The mask for this {@code PagingChunkSource}, which must be a bitmask representing some number of lower + * order bits of a long. */ long mask(); /** *

- * The {@code maxRow} is the greatest possible row which may reference this ChunkSource. This method is used by - * {@link #fillChunkAppend(FillContext, WritableChunk, RowSequence.Iterator)} to determine which of its - * {@code RowSequence} are referencing this {@code PagingChunkSource}. + * The {@code maxRow} is the greatest possible row key which may be referenced in this ChunkSource. This method is + * used by {@link #fillChunkAppend(FillContext, WritableChunk, RowSequence.Iterator)} to determine which of its row + * keys are supplied by this {@code PagingChunkSource}. *

* *

* The default implementation assumes that only one {@code PagingChunkSource} exits for each page reference. That - * is, there is only one {@code PagingChunkSource} for {@code OrderedKey}s with the same bits outside of - * {@link #mask()}. + * is, there is only one {@code PagingChunkSource} for {@link RowSequence RowSequences} with the same bits outside + * of {@link #mask()}. *

* *

@@ -52,35 +51,34 @@ public interface PagingChunkSource extends ChunkSource { * this case, one typically will want to override {@code maxRow}. An example such implementation is * {@link ChunkPage}. * - * @param row Any row contained on this page. - * @return the maximum last row of this page, located in the same way as row. + * @param rowKey Any row key contained by this {@link PagingChunkSource} + * @return The maximum last row key of the page, located in the same way as {@code rowKey] */ - default long maxRow(final long row) { - return row | mask(); + default long maxRow(final long rowKey) { + return rowKey | mask(); } /** *

- * Similar to {@link #fillChunk(FillContext, WritableChunk, RowSequence)}, except that the values from the - * ChunkSource are appended to {@code destination}, rather than placed at the beginning. + * Similar to {@link #fillChunk(FillContext, WritableChunk, RowSequence)}, except that the values are appended to + * {@code destination}, rather than placed at the beginning. *

* *

* The values to fill into {@code destination} are specified by {@code rowSequenceIterator}, whose * {@link RowSequence#firstRowKey()} must exist, and must be represented by this {@code PagingChunkSource} (modulo - * {#link @mask}), otherwise results are undefined. + * {@link #mask()}), otherwise results are undefined. *

* *

- * No more than the elements in {@code rowSequenceIterator}, which are on the same page as - * {@link RowSequence#firstRowKey()}, have their values appended to {@code destination}, and consumed from - * {@code rowSequenceIterator}. Indices are on the same page when the bits outside of {@link #mask()} are identical. + * All values specified by {@code rowSequenceIterator} that are on the same page as its next row key will be + * appended to {@code destination}. Row keys are on the same page when the bits outside of {@link #mask()} are + * identical. * - * @param context A context containing all mutable/state related data used in retrieving the Chunk. In particular, - * the Context may be used to provide a Chunk data pool - * @param destination The chunk to append the results to. - * @param rowSequenceIterator The iterator to the ordered keys, which contain at least the keys to extract from this - * {@code ChunkSource}. The keys to extract will be at the beginning of iteration order. + * @param context A context containing all mutable/state related data used in filling {@code destination} + * @param destination The {@link WritableChunk} to append the results to + * @param rowSequenceIterator An iterator over the remaining row keys specifying the values to retrieve, which + * contains at least the keys to extract from this {@code {PagingChunkSource} */ void fillChunkAppend( @NotNull FillContext context, diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionObject.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionObject.java index 8bad3352397..b52d882f4e5 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionObject.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionObject.java @@ -11,7 +11,6 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; import io.deephaven.engine.page.ChunkPage; -import io.deephaven.engine.page.Page; import io.deephaven.engine.rowset.RowSet; import org.jetbrains.annotations.NotNull; @@ -22,7 +21,7 @@ * {@link ColumnChunkPageStore column chunk page stores}. */ public final class ParquetColumnRegionObject extends ParquetColumnRegionBase - implements ColumnRegionObject, ParquetColumnRegion, Page { + implements ColumnRegionObject, ParquetColumnRegion { private volatile Supplier> dictionaryKeysRegionSupplier; private volatile Supplier> dictionaryValuesRegionSupplier; diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageBoolean.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageBoolean.java index da93873e84b..ea57d014848 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageBoolean.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageBoolean.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageByte.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageByte.java index e90fcd16fbd..47f836fe05f 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageByte.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageByte.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageChar.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageChar.java index cd87587cef4..35276bed24b 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageChar.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageChar.java @@ -43,19 +43,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageDouble.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageDouble.java index 6ada97709d9..74f3a8a6a1c 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageDouble.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageDouble.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageFloat.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageFloat.java index bf11bffb868..887d6b4b342 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageFloat.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageFloat.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageInt.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageInt.java index 1b3e7d05a13..96b17de1c5a 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageInt.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageInt.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageLong.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageLong.java index cb31002defa..4d48a3c2534 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageLong.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageLong.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageObject.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageObject.java index 6b97e66a983..ec7cf91af3a 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageObject.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageObject.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override diff --git a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageShort.java b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageShort.java index cc88fab15aa..2a56e53e9a1 100644 --- a/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageShort.java +++ b/extensions/source-support/src/main/java/io/deephaven/generic/page/ChunkHolderPageShort.java @@ -48,19 +48,19 @@ public final long firstRowOffset() { } @Override - public final long maxRow(final long row) { - return (row & ~mask()) | (firstRowOffset() + storage.length - 1); + public final long maxRow(final long rowKey) { + return (rowKey & ~mask()) | (firstRowOffset() + storage.length - 1); } /** - * @return The offset into the chunk for this row + * @return The offset into the chunk for this row key * @apiNote This function is for convenience over {@link #getRowOffset(long)}, so the caller doesn't have to cast to * an int. - * @implNote This page is known to be backed by chunk, so {@code currentView.size()} is an int, and so is the + * @implNote This page is known to be backed by a chunk, so {@code currentView.size()} is an int, and so is the * offset. */ - private int getChunkOffset(final long row) { - return (int) getRowOffset(row); + private int getChunkOffset(final long rowKey) { + return (int) getRowOffset(rowKey); } @Override From 02fb45a85138a07adf3f95303c6e0e45d0137242 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 29 Dec 2023 22:02:32 -0500 Subject: [PATCH 4/7] Maybe I finally got it right now? --- .../java/io/deephaven/engine/page/Page.java | 23 +++------- .../io/deephaven/engine/page/PageStore.java | 5 ++- .../engine/page/PagingChunkSource.java | 18 +++++++- .../PagingContextHolder.java} | 14 +++--- .../sources/regioned/ColumnRegionByte.java | 3 +- .../sources/regioned/ColumnRegionChar.java | 3 +- .../sources/regioned/ColumnRegionDouble.java | 3 +- .../sources/regioned/ColumnRegionFloat.java | 3 +- .../sources/regioned/ColumnRegionInt.java | 3 +- .../sources/regioned/ColumnRegionLong.java | 3 +- .../sources/regioned/ColumnRegionObject.java | 3 +- .../sources/regioned/ColumnRegionShort.java | 3 +- .../regioned/GenericColumnRegionBase.java | 19 -------- .../sources/regioned/RegionedPageStore.java | 6 --- .../table/pagestore/ColumnChunkPageStore.java | 27 ++++++++---- .../OffsetIndexBasedColumnChunkPageStore.java | 3 ++ .../VariablePageSizeColumnChunkPageStore.java | 2 + .../table/region/ParquetColumnRegionBase.java | 43 ++----------------- 18 files changed, 77 insertions(+), 107 deletions(-) rename engine/table/src/main/java/io/deephaven/engine/{table/impl/sources/regioned/RegionContextHolder.java => page/PagingContextHolder.java} (90%) diff --git a/engine/table/src/main/java/io/deephaven/engine/page/Page.java b/engine/table/src/main/java/io/deephaven/engine/page/Page.java index 3078a69ccbe..1961affba3b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/Page.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/Page.java @@ -14,24 +14,13 @@ import org.jetbrains.annotations.NotNull; /** - * This provides the {@link ChunkSource} interface to a contiguous block of data beginning at {@link #firstRowOffset()} - * and continuing to some row less than or equal to {@link #firstRowOffset()} + {@link #maxRow(long)}. + * Pages are {@link PagingChunkSource PagingChunkSources} that can supply values from a subset of a contiguous block of + * row key space beginning at {@link #firstRowOffset()} and continuing to {@link #firstRowOffset()} + + * {@link #maxRow(long)}. Not all row keys within the range may be valid; that is, pages may be sparse. *

- * Non overlapping pages can be collected together in a {@link PageStore}, which provides the {@link ChunkSource} - * interface to the collection of all of its Pages. - *

- * There are two distinct use cases/types of pages. The first use case are Pages which always have a length() - * > 0. These store length() values, which can be assessed via the {@link ChunkSource} methods. Valid - * {@link RowSequence row sequences} passed to those methods will have their offset in the range [firstRowOffset(), firstRowOffset() + - * length()). Passing OrderKeys with offsets outside of this range will have undefined results. - *

- * The second use case will always have length() == 0 and firstRowOffset() == 0. These represent "Null" regions which - * return a fixed value, typically a null value, for every {@link RowSequence} passed into the {@link ChunkSource} - * methods. In order to have this use case, override {@code length} and override {@code lastRow} as {@code maxRow}. - *

- * Though the {@link ChunkSource} methods ignore the non-offset portion of the rows in the {@link RowSequence}, they can - * assume they are identical for all the passed in elements of the {@link RowSequence}. For instance, they can use the - * simple difference between the complete row value to determine a length. + * Pages may be held within one or more {@link PageStore} instances. The PageStore is responsible for determining which + * row keys in absolute space are mapped to a particular Page. Pages need only concern themselves with lower order bits + * of the row keys they are asked for, after applying their {@link #mask()}. */ public interface Page extends PagingChunkSource { diff --git a/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java b/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java index 76ebe61acf1..739847aebb7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PageStore.java @@ -15,8 +15,9 @@ import org.jetbrains.annotations.Nullable; /** - * PageStores are a collection of non-overlapping pages, which provides a single {@link ChunkSource} interface across - * all the pages. + * PageStores are a collection of non-overlapping {@link Page Pages}, providing a single {@link PagingChunkSource} + * across all the pages. PageStores are responsible for mapping row keys to pages. PageStores may themselves be Pages + * nested within other PageStores. */ public interface PageStore> extends PagingChunkSource, DefaultChunkSource.SupportsContiguousGet { diff --git a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java index 5708c96c690..4853ae5ec13 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java @@ -7,7 +7,11 @@ import io.deephaven.engine.table.ChunkSource; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.table.SharedContext; +import io.deephaven.engine.table.impl.DefaultChunkSource; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; /** * In order to be able to cache and reuse {@link ChunkSource ChunkSources} across multiple tables (or other references), @@ -20,8 +24,20 @@ *

* Also, a new method {@link #fillChunkAppend(FillContext, WritableChunk, RowSequence.Iterator)} is added, which * supports filling a chunk incrementally across a series of pages. + *

+ * In order to support arbitrary nesting and re-use of {@link PagingChunkSource} implementations, it is required that + * all implementations use or extend {@link io.deephaven.engine.table.impl.DefaultGetContext DefaultGetContext} and + * {@link PagingContextHolder} as their {@link #makeGetContext(int, SharedContext) GetContext} and + * {@link #makeFillContext(int, SharedContext) FillContext}, respectively. Nested implementations may thus store their + * own state via the {@link PagingContextHolder#getInnerContext() inner context}, using sub-classes of + * {@link PagingContextHolder} to support chaining of nested state. */ -public interface PagingChunkSource extends ChunkSource { +public interface PagingChunkSource extends DefaultChunkSource { + + @Override + default FillContext makeFillContext(final int chunkCapacity, @Nullable final SharedContext sharedContext) { + return new PagingContextHolder(chunkCapacity, sharedContext); + } /** * This mask is applied to {@link RowSequence RowSequences} which are passed into {@link #getChunk}, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java b/engine/table/src/main/java/io/deephaven/engine/page/PagingContextHolder.java similarity index 90% rename from engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java rename to engine/table/src/main/java/io/deephaven/engine/page/PagingContextHolder.java index 829d14099e6..61c552d2d4e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionContextHolder.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PagingContextHolder.java @@ -1,7 +1,7 @@ /** * Copyright (c) 2016-2022 Deephaven Data Labs and Patent Pending */ -package io.deephaven.engine.table.impl.sources.regioned; +package io.deephaven.engine.page; import io.deephaven.engine.table.ChunkSource; import io.deephaven.engine.table.Context; @@ -11,18 +11,18 @@ import org.jetbrains.annotations.Nullable; /** - * {@link ChunkSource.FillContext} implementation for use by {@link RegionedPageStore} implementations. This is + * {@link ChunkSource.FillContext} implementation for use by {@link PagingChunkSource} implementations. This is * basically a re-usable box around an inner {@link Context context} object, filled with whatever the most recently used - * region chose to store. + * {@link Page} chose to store. */ -public class RegionContextHolder implements ChunkSource.FillContext { +public class PagingContextHolder implements ChunkSource.FillContext { private final int chunkCapacity; private final SharedContext sharedContext; private Context innerContext; - public RegionContextHolder(final int chunkCapacity, @Nullable final SharedContext sharedContext) { + public PagingContextHolder(final int chunkCapacity, @Nullable final SharedContext sharedContext) { this.chunkCapacity = chunkCapacity; this.sharedContext = sharedContext; } @@ -30,8 +30,8 @@ public RegionContextHolder(final int chunkCapacity, @Nullable final SharedContex /** * {@inheritDoc} * - * @implNote This implementation always returns {@code true}, as the {@link RegionedPageStore} fill implementation - * follows an append pattern over multiple regions when necessary, and all known inner context + * @implNote This implementation always returns {@code true}, as the {@link PageStore} fill implementation follows + * an append pattern over multiple {@link Page pages} when necessary, and all known inner context * implementations trivially support unbounded fill. We thus make this a requirement for future inner * context implementations, either naturally or via a slicing/looping pattern. */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionByte.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionByte.java index a8f00446713..87e2a3064ee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionByte.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionByte.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -28,7 +29,7 @@ public interface ColumnRegionByte extends ColumnRegion { /** * Get a single byte from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The byte value at the specified element row ket diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionChar.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionChar.java index f990541652e..944d0d550b1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionChar.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionChar.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -26,7 +27,7 @@ public interface ColumnRegionChar extends ColumnRegion { /** * Get a single char from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The char value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionDouble.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionDouble.java index a3445cf2a89..a2a77240898 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionDouble.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionDouble.java @@ -11,6 +11,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -31,7 +32,7 @@ public interface ColumnRegionDouble extends ColumnRegion /** * Get a single double from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The double value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionFloat.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionFloat.java index b8cc6489044..c16ac87565c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionFloat.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionFloat.java @@ -11,6 +11,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -31,7 +32,7 @@ public interface ColumnRegionFloat extends ColumnRegion /** * Get a single float from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The float value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionInt.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionInt.java index 44e7863f418..e969bb28c56 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionInt.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionInt.java @@ -11,6 +11,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -31,7 +32,7 @@ public interface ColumnRegionInt extends ColumnRegion { /** * Get a single int from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The int value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionLong.java index a1cf5ee8856..de33fea447f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionLong.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionLong.java @@ -11,6 +11,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -31,7 +32,7 @@ public interface ColumnRegionLong extends ColumnRegion { /** * Get a single long from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The long value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionObject.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionObject.java index 606431ae788..26b25270ff9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionObject.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionObject.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.chunk.attributes.Any; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; @@ -35,7 +36,7 @@ public interface ColumnRegionObject extends ColumnR /** * Get a single object from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The object value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionShort.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionShort.java index 588f34eeb7c..eeef5380f32 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionShort.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegionShort.java @@ -11,6 +11,7 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; @@ -31,7 +32,7 @@ public interface ColumnRegionShort extends ColumnRegion /** * Get a single short from this region. * - * @param context A {@link RegionContextHolder} to enable resource caching where suitable, with current + * @param context A {@link PagingContextHolder} to enable resource caching where suitable, with current * region index pointing to this region * @param elementIndex Element row key in the table's address space * @return The short value at the specified element row key diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java index 93185889d28..5f24d1737bb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/GenericColumnRegionBase.java @@ -4,9 +4,7 @@ package io.deephaven.engine.table.impl.sources.regioned; import io.deephaven.chunk.attributes.Any; -import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; -import org.jetbrains.annotations.Nullable; /** * Base {@link ColumnRegion} implementation. @@ -35,21 +33,4 @@ protected final void throwIfInvalidated() { throw new InvalidatedRegionException("Column region has been invalidated due to data removal"); } } - - @Override - public final FillContext makeFillContext(final int chunkCapacity, @Nullable final SharedContext sharedContext) { - throw new UnsupportedOperationException( - "ColumnRegions do not support makeFillContext. Implementations should accept " - + "io.deephaven.engine.table.impl.sources.regioned.RegionContextHolder and get/set/update " - + "their own context as needed."); - } - - @Override - public final GetContext makeGetContext(final int chunkCapacity, @Nullable final SharedContext sharedContext) { - throw new UnsupportedOperationException( - "ColumnRegions do not support makeGetContext. Implementations should accept " - + "io.deephaven.engine.table.impl.DefaultGetContext with a " - + "io.deephaven.engine.table.impl.sources.regioned.RegionContextHolder as FillContext, and " - + "get/set/update their own context as needed."); - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java index 5f28b010cb7..7893e456dbc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPageStore.java @@ -6,7 +6,6 @@ import io.deephaven.base.MathUtil; import io.deephaven.base.verify.Require; import io.deephaven.chunk.attributes.Any; -import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.page.Page; import io.deephaven.engine.page.PageStore; import io.deephaven.engine.rowset.RowSequence; @@ -91,11 +90,6 @@ default REGION_TYPE getPageContaining(@Nullable final FillContext fillContext, f return lookupRegion(rowKey); } - @Override - default FillContext makeFillContext(final int chunkCapacity, final SharedContext sharedContext) { - return new RegionContextHolder(chunkCapacity, sharedContext); - } - /** * Class to calculate and encapsulate the parameters of a RegionedPageStore. */ diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java index 4f93849f460..3e7d2566bec 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java @@ -4,9 +4,10 @@ package io.deephaven.parquet.table.pagestore; import io.deephaven.base.verify.Require; +import io.deephaven.engine.page.PagingContextHolder; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Context; -import io.deephaven.engine.table.impl.DefaultGetContext; +import io.deephaven.engine.table.SharedContext; import io.deephaven.parquet.table.pagestore.topage.ToPage; import io.deephaven.engine.table.Releasable; import io.deephaven.chunk.attributes.Any; @@ -170,15 +171,25 @@ public boolean usesDictionaryOnEveryPage() { @Override public void close() {} - public boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { + FillContext innerFillContext(@NotNull final FillContext context) { + // Call this method from the appropriate place in the implementation of getPageContaining + return ((PagingContextHolder) context) + .updateInnerContext(this::fillContextUpdater); + } + + private boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { + // Replace this with a test to see if the fill context comes from this.ColumnChunkReader return currentInnerContext == DEFAULT_FILL_INSTANCE; } - public boolean isGetContextCompatible(@Nullable final Context currentInnerContext) { - if (!(currentInnerContext instanceof DefaultGetContext)) { - return false; - } - final DefaultGetContext getContext = (DefaultGetContext) currentInnerContext; - return isFillContextCompatible(getContext.getFillContext()); + private T fillContextUpdater( + int chunkCapacity, + @Nullable final SharedContext sharedContext, + @Nullable final Context currentInnerContext) { + // noinspection unchecked + return (T) (isFillContextCompatible(currentInnerContext) + ? currentInnerContext + // Replace this with getting a context from this.ColumnChunkReader + : makeFillContext(chunkCapacity, sharedContext)); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java index 71e9ffbe4fa..eb230a8e648 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java @@ -115,6 +115,8 @@ private ChunkPage getPage(final int pageNum) { synchronized (pageState) { // Make sure no one materialized this page as we waited for the lock if ((localRef = pageState.pageRef) == null || (page = localRef.get()) == null) { + // getPage() should accept the outer fill context, and get an inner fill context from + // this.ColumnChunkReader to pass into getPageReader. final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); try { page = new PageCache.IntrusivePage<>(toPage(offsetIndex.getFirstRowIndex(pageNum), reader)); @@ -148,6 +150,7 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = (numPages - 1); } } + // Turn return getPage(pageNum); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java index ee8239c38cb..654fb8dafbc 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java @@ -43,6 +43,8 @@ final class VariablePageSizeColumnChunkPageStore extends Colum pageRowOffsets = new long[INIT_ARRAY_SIZE + 1]; pageRowOffsets[0] = 0; columnPageReaders = new ColumnPageReader[INIT_ARRAY_SIZE]; + // We probably need a super-interface of Iterator to allow ourselves to set or clear the inner fill context + // to be used by next. columnPageReaderIterator = columnChunkReader.getPageIterator(); // noinspection unchecked diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java index b7dee8e71c2..2b135085501 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/region/ParquetColumnRegionBase.java @@ -4,11 +4,7 @@ package io.deephaven.parquet.table.region; import io.deephaven.base.verify.Require; -import io.deephaven.engine.table.Context; -import io.deephaven.engine.table.SharedContext; -import io.deephaven.engine.table.impl.DefaultGetContext; import io.deephaven.engine.table.impl.sources.regioned.GenericColumnRegionBase; -import io.deephaven.engine.table.impl.sources.regioned.RegionContextHolder; import io.deephaven.parquet.table.pagestore.ColumnChunkPageStore; import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.Chunk; @@ -16,7 +12,6 @@ import io.deephaven.engine.page.ChunkPage; import io.deephaven.engine.rowset.RowSequence; import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; import javax.annotation.OverridingMethodsMustInvokeSuper; @@ -41,7 +36,7 @@ public final Chunk getChunk( @NotNull final GetContext context, @NotNull final RowSequence rowSequence) { throwIfInvalidated(); - return columnChunkPageStore.getChunk(innerGetContext(context), rowSequence); + return columnChunkPageStore.getChunk(context, rowSequence); } @Override @@ -50,7 +45,7 @@ public final Chunk getChunk( final long firstKey, final long lastKey) { throwIfInvalidated(); - return columnChunkPageStore.getChunk(innerGetContext(context), firstKey, lastKey); + return columnChunkPageStore.getChunk(context, firstKey, lastKey); } @Override @@ -59,7 +54,7 @@ public final void fillChunk( @NotNull final WritableChunk destination, @NotNull final RowSequence rowSequence) { throwIfInvalidated(); - columnChunkPageStore.fillChunk(innerFillContext(context), destination, rowSequence); + columnChunkPageStore.fillChunk(context, destination, rowSequence); } @Override @@ -68,7 +63,7 @@ public final void fillChunkAppend( @NotNull final WritableChunk destination, @NotNull final RowSequence.Iterator rowSequenceIterator) { throwIfInvalidated(); - columnChunkPageStore.fillChunkAppend(innerFillContext(context), destination, rowSequenceIterator); + columnChunkPageStore.fillChunkAppend(context, destination, rowSequenceIterator); } @Override @@ -83,34 +78,4 @@ public void releaseCachedResources() { ParquetColumnRegion.super.releaseCachedResources(); columnChunkPageStore.releaseCachedResources(); } - - private FillContext innerFillContext(@NotNull final FillContext context) { - return ((RegionContextHolder) context) - .updateInnerContext(this::fillContextUpdater); - } - - private T fillContextUpdater( - int chunkCapacity, - @Nullable final SharedContext sharedContext, - @Nullable final Context currentInnerContext) { - // noinspection unchecked - return (T) (columnChunkPageStore.isFillContextCompatible(currentInnerContext) - ? currentInnerContext - : columnChunkPageStore.makeFillContext(chunkCapacity, sharedContext)); - } - - private GetContext innerGetContext(@NotNull final GetContext context) { - return ((RegionContextHolder) DefaultGetContext.getFillContext(context)) - .updateInnerContext(this::getContextUpdater); - } - - private T getContextUpdater( - int chunkCapacity, - @Nullable final SharedContext sharedContext, - @Nullable final Context currentInnerContext) { - // noinspection unchecked - return (T) (columnChunkPageStore.isGetContextCompatible(currentInnerContext) - ? currentInnerContext - : columnChunkPageStore.makeGetContext(chunkCapacity, sharedContext)); - } } From 3dee6c2d2d1adcd4be5de2a2ee1070f1cc91ef7e Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 29 Dec 2023 22:08:26 -0500 Subject: [PATCH 5/7] Replication and spotless cleanup --- .../time/src/main/java/io/deephaven/time/DateTimeUtils.java | 4 ++-- extensions/parquet/table/build.gradle | 1 + .../parquet/table/transfer/LocalDateTimeArrayTransfer.java | 3 +-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java index 56a49c8c2ef..aaffc736e1f 100644 --- a/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java +++ b/engine/time/src/main/java/io/deephaven/time/DateTimeUtils.java @@ -2990,8 +2990,8 @@ public static Instant upperBin(@Nullable final Instant instant, long intervalNan * * @param dateTime zoned date time for which to evaluate the start of the containing window * @param intervalNanos size of the window in nanoseconds - * @param offset The window start offset in nanoseconds. - * For example, a value of MINUTE would offset all windows by one minute. + * @param offset The window start offset in nanoseconds. For example, a value of MINUTE would offset all windows by + * one minute. * @return {@code null} if either input is {@code null}; otherwise, a {@link ZonedDateTime} representing the end of * the window */ diff --git a/extensions/parquet/table/build.gradle b/extensions/parquet/table/build.gradle index 98a40a10546..232f9b69b73 100644 --- a/extensions/parquet/table/build.gradle +++ b/extensions/parquet/table/build.gradle @@ -75,6 +75,7 @@ spotless { '**/*Float*.java', '**/*Double*.java', '**/*Object*.java', + '**/*Transfer*.java', ) } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/LocalDateTimeArrayTransfer.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/LocalDateTimeArrayTransfer.java index 4b1e8a74da4..f625be53321 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/LocalDateTimeArrayTransfer.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/transfer/LocalDateTimeArrayTransfer.java @@ -16,8 +16,7 @@ import java.nio.LongBuffer; import java.time.LocalDateTime; -final class LocalDateTimeArrayTransfer - extends PrimitiveArrayAndVectorTransfer { +final class LocalDateTimeArrayTransfer extends PrimitiveArrayAndVectorTransfer { // We encode LocalDateTime as primitive longs LocalDateTimeArrayTransfer(@NotNull final ColumnSource columnSource, @NotNull final RowSequence tableRowSet, final int targetPageSizeInBytes) { From e673959b46da2e2d3c660c3c354ee8e643761234 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sun, 31 Dec 2023 22:37:57 -0500 Subject: [PATCH 6/7] Fix JavaDoc oops --- .../main/java/io/deephaven/engine/page/PagingChunkSource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java index 4853ae5ec13..98a82040c03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java @@ -68,7 +68,7 @@ default FillContext makeFillContext(final int chunkCapacity, @Nullable final Sha * {@link ChunkPage}. * * @param rowKey Any row key contained by this {@link PagingChunkSource} - * @return The maximum last row key of the page, located in the same way as {@code rowKey] + * @return The maximum last row key of the page, located in the same way as {@code rowKey} */ default long maxRow(final long rowKey) { return rowKey | mask(); @@ -94,7 +94,7 @@ default long maxRow(final long rowKey) { * @param context A context containing all mutable/state related data used in filling {@code destination} * @param destination The {@link WritableChunk} to append the results to * @param rowSequenceIterator An iterator over the remaining row keys specifying the values to retrieve, which - * contains at least the keys to extract from this {@code {PagingChunkSource} + * contains at least the keys to extract from this {@code PagingChunkSource} */ void fillChunkAppend( @NotNull FillContext context, From 81a0910a52b6f0c48fba2c4029a665f8328d1e10 Mon Sep 17 00:00:00 2001 From: Shivam Malhotra Date: Wed, 3 Jan 2024 22:50:50 +0530 Subject: [PATCH 7/7] Replaced comments related to deephaven-core#4836 with TODOs --- .../java/io/deephaven/engine/page/PagingChunkSource.java | 1 - .../parquet/table/pagestore/ColumnChunkPageStore.java | 8 +++++--- .../pagestore/OffsetIndexBasedColumnChunkPageStore.java | 5 ++--- .../pagestore/VariablePageSizeColumnChunkPageStore.java | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java index 98a82040c03..d1d51ea7f41 100644 --- a/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/page/PagingChunkSource.java @@ -9,7 +9,6 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.SharedContext; import io.deephaven.engine.table.impl.DefaultChunkSource; -import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java index 3e7d2566bec..c91911de204 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/ColumnChunkPageStore.java @@ -172,13 +172,15 @@ public boolean usesDictionaryOnEveryPage() { public void close() {} FillContext innerFillContext(@NotNull final FillContext context) { - // Call this method from the appropriate place in the implementation of getPageContaining + // TODO(deephaven-core#4836): Call this method from the appropriate place in the implementation of + // getPageContaining to populate the context object return ((PagingContextHolder) context) .updateInnerContext(this::fillContextUpdater); } private boolean isFillContextCompatible(@Nullable final Context currentInnerContext) { - // Replace this with a test to see if the fill context comes from this.ColumnChunkReader + // TODO(deephaven-core#4836): Replace this with a test to see if the fill context comes from + // this.ColumnChunkReader return currentInnerContext == DEFAULT_FILL_INSTANCE; } @@ -189,7 +191,7 @@ private T fillContextUpdater( // noinspection unchecked return (T) (isFillContextCompatible(currentInnerContext) ? currentInnerContext - // Replace this with getting a context from this.ColumnChunkReader + // TODO(deephaven-core#4836): Replace this with getting a context from this.ColumnChunkReader : makeFillContext(chunkCapacity, sharedContext)); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java index eb230a8e648..7518470d8fe 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/OffsetIndexBasedColumnChunkPageStore.java @@ -115,8 +115,8 @@ private ChunkPage getPage(final int pageNum) { synchronized (pageState) { // Make sure no one materialized this page as we waited for the lock if ((localRef = pageState.pageRef) == null || (page = localRef.get()) == null) { - // getPage() should accept the outer fill context, and get an inner fill context from - // this.ColumnChunkReader to pass into getPageReader. + // TODO(deephaven-core#4836): getPage() should accept the outer fill context, and get an inner fill + // context from this.ColumnChunkReader to pass into getPageReader. final ColumnPageReader reader = columnPageDirectAccessor.getPageReader(pageNum); try { page = new PageCache.IntrusivePage<>(toPage(offsetIndex.getFirstRowIndex(pageNum), reader)); @@ -150,7 +150,6 @@ public ChunkPage getPageContaining(@Nullable final FillContext fillContext pageNum = (numPages - 1); } } - // Turn return getPage(pageNum); } } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java index 654fb8dafbc..df27c76ba38 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/pagestore/VariablePageSizeColumnChunkPageStore.java @@ -43,8 +43,8 @@ final class VariablePageSizeColumnChunkPageStore extends Colum pageRowOffsets = new long[INIT_ARRAY_SIZE + 1]; pageRowOffsets[0] = 0; columnPageReaders = new ColumnPageReader[INIT_ARRAY_SIZE]; - // We probably need a super-interface of Iterator to allow ourselves to set or clear the inner fill context - // to be used by next. + // TODO(deephaven-core#4836): We probably need a super-interface of Iterator to allow ourselves to set or clear + // the inner fill context to be used by next. columnPageReaderIterator = columnChunkReader.getPageIterator(); // noinspection unchecked