diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/WritableBooleanChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/WritableBooleanChunk.java index 6cf87f9c84f..9c442bd7ec1 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/WritableBooleanChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/WritableBooleanChunk.java @@ -10,7 +10,9 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.util.pools.MultiChunkPool; +import io.deephaven.function.ArraySort; import io.deephaven.util.type.TypeUtils; +import org.jetbrains.annotations.NotNull; import java.util.Arrays; // region FillWithNullValueImports diff --git a/engine/chunk/src/main/java/io/deephaven/chunk/WritableObjectChunk.java b/engine/chunk/src/main/java/io/deephaven/chunk/WritableObjectChunk.java index e61cb49793a..066aa94e193 100644 --- a/engine/chunk/src/main/java/io/deephaven/chunk/WritableObjectChunk.java +++ b/engine/chunk/src/main/java/io/deephaven/chunk/WritableObjectChunk.java @@ -11,6 +11,8 @@ import io.deephaven.chunk.util.pools.MultiChunkPool; import io.deephaven.function.ArraySort; +import io.deephaven.util.type.TypeUtils; +import org.jetbrains.annotations.NotNull; import java.util.Arrays; // region FillWithNullValueImports diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContext.java index 20295bd0872..77e25fc22f6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContext.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContext.java @@ -3,36 +3,25 @@ // package io.deephaven.engine.table.impl; -import io.deephaven.api.Strings; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.liveness.LivenessScopeStack; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.TrackingRowSet; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; import io.deephaven.engine.table.impl.select.*; -import io.deephaven.engine.table.impl.sources.NullValueColumnSource; -import io.deephaven.engine.util.TableTools; import io.deephaven.util.SafeCloseable; -import org.jetbrains.annotations.MustBeInvokedByOverriders; import java.util.List; -import java.util.Map; /** * Base class for {@link PushdownFilterContext} to help with execution cost tracking. */ -public class BasePushdownFilterContext implements PushdownFilterContext { +public interface BasePushdownFilterContext extends PushdownFilterContext { /** * Enum for the behavior of a filter when applied to null values. */ - public enum FilterNullBehavior { + enum FilterNullBehavior { /** * The filter includes nulls in its results, like {@code x == null}. */ @@ -49,270 +38,58 @@ public enum FilterNullBehavior { FAILS_ON_NULLS } - protected final WhereFilter filter; - private final List> columnSources; - - private final boolean isRangeFilter; - private final boolean isMatchFilter; - private final boolean supportsDictionaryFiltering; - - private long executedFilterCost; - - /** - * The behavior of this filter when applied to null values. This is lazily computed on first access. Should be - * accessed via {@link #filterNullBehavior()}. - */ - private volatile FilterNullBehavior filterNullBehavior; - - /** - * A dummy table to use for initializing {@link ConditionFilter}. This is lazily computed on first access. Should be - * accessed via {@link #conditionalFilterInitTable()}. - */ - private volatile Table conditionalFilterInitTable; - /** * Interface for a unified chunk filter that can be used to apply a filter to a chunk of data, whether the * underlying filter is a {@link ExposesChunkFilter} or a {@link ConditionFilter}. */ - public interface UnifiedChunkFilter extends SafeCloseable { + interface UnifiedChunkFilter extends SafeCloseable { LongChunk filter(Chunk values, LongChunk keys); } - public BasePushdownFilterContext( - final WhereFilter filter, - final List> columnSources) { - if (!filter.permitParallelization()) { - throw new IllegalArgumentException( - "filter must be stateless, but does not permit parallelization: " + filter); - } - this.filter = filter; - this.columnSources = columnSources; - - executedFilterCost = 0; - - isRangeFilter = filter instanceof RangeFilter - && ((RangeFilter) filter).getRealFilter() instanceof AbstractRangeFilter; - isMatchFilter = filter instanceof MatchFilter && - ((MatchFilter) filter).getFailoverFilterIfCached() == null; - final boolean isConditionFilter = filter instanceof ConditionFilter; - - // TODO (DH-19666): Multi column filters are not supported yet - // Do not use columnSources.size(), multiple logical columns may alias (rename) the same physical column, - // yielding a single entry. - supportsDictionaryFiltering = (isRangeFilter || isMatchFilter - || (isConditionFilter && ((ConditionFilter) filter).getNumInputsUsed() == 1)) - && ((filter instanceof ExposesChunkFilter && ((ExposesChunkFilter) filter).chunkFilter().isPresent()) - || isConditionFilter); - - filterNullBehavior = null; // lazily initialized - } - - @Override - public long executedFilterCost() { - return executedFilterCost; - } - - @Override - public void updateExecutedFilterCost(long executedFilterCost) { - this.executedFilterCost = executedFilterCost; - } + /** + * Gets the filter associated with this context. + */ + WhereFilter filter(); /** * Get the column sources this filter will use. */ - public List> columnSources() { - return columnSources; - } + List> columnSources(); /** - * Whether this filter supports parquet dictionary filtering, which necessitates direct chunk filtering, i.e., it - * can be applied to a chunk of data rather than a table. This includes any filter that implements {#@link - * ExposesChunkFilter} or {@link ConditionFilter} with exactly one column. + * Whether this filter supports direct chunk filtering, i.e., it can be applied to a chunk of data rather than a + * table. This includes any filter that implements {@link ExposesChunkFilter} or {@link ConditionFilter} with + * exactly one column. */ - public final boolean supportsDictionaryFiltering() { - return supportsDictionaryFiltering; - } + boolean supportsChunkFiltering(); /** * Whether this filter supports filtering based on parquet metadata. */ - public final boolean supportsMetadataFiltering() { - return isRangeFilter || isMatchFilter; - } + boolean supportsMetadataFiltering(); - public final boolean supportsInMemoryDataIndexFiltering() { - // Note: if there is a cheap way to check if the filter will never be applicable for an in-memory data index, we - // would like to add that check here. - return true; - } + boolean supportsInMemoryDataIndexFiltering(); - public final boolean supportsDeferredDataIndexFiltering() { - // Note: if there is a cheap way to check if the filter will never be applicable for a deferred data index, we - // would like to add that check here. - return true; - } + boolean supportsDeferredDataIndexFiltering(); /** * The filter to use for parquet metadata filtering. Can only call when {@link #supportsMetadataFiltering()} is * {@code true}. */ - public final WhereFilter filterForMetadataFiltering() { - if (isRangeFilter) { - return ((RangeFilter) filter).getRealFilter(); - } - if (isMatchFilter) { - return filter; - } - throw new IllegalStateException("Should only use when supportsMetadataFiltering is true"); - } + WhereFilter filterForMetadataFiltering(); /** * Get the behavior of this filter when applied to null values. This is lazily computed on first access. */ - public FilterNullBehavior filterNullBehavior() { - FilterNullBehavior local = filterNullBehavior; - if (local == null) { - synchronized (this) { - local = filterNullBehavior; - if (local == null) { - local = computeFilterNullBehavior(); - filterNullBehavior = local; - } - } - } - return local; - } - - private FilterNullBehavior computeFilterNullBehavior() { - // Create a dummy table with a single row and column, and `null` entry, and apply the filter to see - // if the filter includes nulls. - final ColumnSource columnSource = columnSources.get(0); - final NullValueColumnSource nullValueColumnSource = - NullValueColumnSource.getInstance(columnSource.getType(), columnSource.getComponentType()); - final Map> columnSourceMap = - Map.of(filter.getColumns().get(0), nullValueColumnSource); - try (final SafeCloseable ignored = LivenessScopeStack.open()) { - final Table nullTestDummyTable = TableTools.newTable(1, columnSourceMap); - final TrackingRowSet rowSet = nullTestDummyTable.getRowSet(); - try (final RowSet result = filter.filter(rowSet, rowSet, nullTestDummyTable, false)) { - return result.isEmpty() ? FilterNullBehavior.EXCLUDES_NULLS : FilterNullBehavior.INCLUDES_NULLS; - } catch (final Exception e) { - return FilterNullBehavior.FAILS_ON_NULLS; - } - } - } + FilterNullBehavior filterNullBehavior(); /** * Create a {@link UnifiedChunkFilter} for the {@link WhereFilter} that efficiently filters chunks of data. Every * thread that uses this should create its own instance and must close it after use. Can only call when - * {@link #supportsDictionaryFiltering()} is {@code true} + * {@link #supportsChunkFiltering()} is {@code true} * * @param maxChunkSize the maximum size of the chunk that will be filtered * @return the initialized {@link UnifiedChunkFilter} */ - public final UnifiedChunkFilter createChunkFilter(final int maxChunkSize) { - if (!supportsDictionaryFiltering) { - throw new IllegalStateException("Filter does not support chunk filtering: " + Strings.of(filter)); - } - if (filter instanceof ExposesChunkFilter) { - final ChunkFilter chunkFilter = ((ExposesChunkFilter) filter).chunkFilter() - .orElseThrow(() -> new IllegalStateException("ExposesChunkFilter#chunkFilter() returned null.")); - return new DirectChunkFilter(chunkFilter, maxChunkSize); - } else if (filter instanceof ConditionFilter) { - // Create a dummy table with no rows and single column of the correct type and name as the filter. This is - // used to extract a chunk filter kernel from the conditional filter and bind it to the correct name and - // type without capturing references to the actual table or its column sources. - final Table initTable = conditionalFilterInitTable(); - try { - final ConditionFilter conditionFilter = (ConditionFilter) filter; - final AbstractConditionFilter.Filter acfFilter = - conditionFilter.getFilter(initTable, initTable.getRowSet()); - return new ConditionKernelChunkFilter(acfFilter, maxChunkSize); - } catch (final Exception e) { - throw new IllegalArgumentException("Error creating condition filter in BasePushdownFilterContext", e); - } - } else { - throw new UnsupportedOperationException( - "Filter does not support chunk filtering: " + Strings.of(filter)); - } - } - - private Table conditionalFilterInitTable() { - Table local = conditionalFilterInitTable; - if (local == null) { - synchronized (this) { - local = conditionalFilterInitTable; - if (local == null) { - final Map> columnSourceMap = Map.of(filter.getColumns().get(0), - NullValueColumnSource.getInstance( - columnSources.get(0).getType(), - columnSources.get(0).getComponentType())); - local = TableTools.newTable(0, columnSourceMap); - conditionalFilterInitTable = local; - } - } - } - return local; - } - - - /** - * A {@link UnifiedChunkFilter} that wraps a {@link ChunkFilter} directly. - */ - private static final class DirectChunkFilter implements UnifiedChunkFilter { - private final ChunkFilter chunkFilter; - private final WritableLongChunk resultChunk; - - private DirectChunkFilter(final ChunkFilter chunkFilter, final int maxChunkSize) { - this.chunkFilter = chunkFilter; - // We need to create a WritableLongChunk to hold the results of the chunk filter. - this.resultChunk = WritableLongChunk.makeWritableChunk(maxChunkSize); - } - - @Override - public LongChunk filter(Chunk values, LongChunk keys) { - chunkFilter.filter(values, keys, resultChunk); - return resultChunk; - } - - @Override - public void close() { - resultChunk.close(); - } - } - - /** - * A {@link UnifiedChunkFilter} that wraps a {@link ConditionFilter} by extracting its kernel and context. - */ - private static final class ConditionKernelChunkFilter implements UnifiedChunkFilter { - private final AbstractConditionFilter.Filter acfFilter; - private final ConditionFilter.FilterKernel.Context conditionFilterContext; - - private ConditionKernelChunkFilter( - final AbstractConditionFilter.Filter acfFilter, - final int maxChunkSize) { - this.acfFilter = acfFilter; - // Create the context for the ConditionFilter, which will be used to filter chunks. - this.conditionFilterContext = acfFilter.getContext(maxChunkSize); - } - - @Override - public LongChunk filter( - Chunk values, - LongChunk keys) { - // noinspection unchecked - return acfFilter.filter(conditionFilterContext, keys, new Chunk[] {values}); - } - - @Override - public void close() { - conditionFilterContext.close(); - } - } - - @MustBeInvokedByOverriders - @Override - public void close() { - conditionalFilterInitTable = null; - } + UnifiedChunkFilter createChunkFilter(final int maxChunkSize); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContextImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContextImpl.java new file mode 100644 index 00000000000..707e054fa07 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/BasePushdownFilterContextImpl.java @@ -0,0 +1,303 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl; + +import io.deephaven.api.Strings; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.TrackingRowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.select.*; +import io.deephaven.engine.table.impl.sources.NullValueColumnSource; +import io.deephaven.engine.util.TableTools; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.MustBeInvokedByOverriders; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * Base class for {@link BasePushdownFilterContext} to help with execution cost tracking. + */ +public abstract class BasePushdownFilterContextImpl implements BasePushdownFilterContext { + + private final WhereFilter filter; + + private final List> columnSources; + + private final boolean isRangeFilter; + private final boolean isMatchFilter; + private final boolean supportsChunkFiltering; + + private long executedFilterCost; + + /** + * The behavior of this filter when applied to null values. This is lazily computed on first access. Should be + * accessed via {@link #filterNullBehavior()}. + */ + private volatile FilterNullBehavior filterNullBehavior; + + /** + * A dummy table to use for initializing {@link ConditionFilter}. This is lazily computed on first access. Should be + * accessed via {@link #conditionalFilterInitTable()}. + */ + private volatile Table conditionalFilterInitTable; + + public BasePushdownFilterContextImpl( + final WhereFilter filter, + final List> columnSources) { + if (!filter.permitParallelization()) { + throw new IllegalArgumentException( + "filter must be stateless, but does not permit parallelization: " + filter); + } + + this.filter = filter; + this.columnSources = columnSources; + + executedFilterCost = 0; + + isRangeFilter = filter instanceof RangeFilter + && ((RangeFilter) filter).getRealFilter() instanceof AbstractRangeFilter; + isMatchFilter = filter instanceof MatchFilter && + ((MatchFilter) filter).getFailoverFilterIfCached() == null; + + final Optional chunkFilter = ExposesChunkFilter.chunkFilter(filter); + supportsChunkFiltering = chunkFilter.isPresent() + || (filter instanceof ConditionFilter && ((ConditionFilter) filter).getNumInputsUsed() == 1); + + conditionalFilterInitTable = null; // lazily initialized + filterNullBehavior = null; // lazily initialized + } + + @Override + public long executedFilterCost() { + return executedFilterCost; + } + + @Override + public void updateExecutedFilterCost(long executedFilterCost) { + this.executedFilterCost = executedFilterCost; + } + + /** + * Gets the filter associated with this context. + */ + @Override + public final WhereFilter filter() { + return filter; + } + + /** + * Get the column sources this filter will use. + */ + @Override + public final List> columnSources() { + return columnSources; + } + + /** + * Whether this filter supports direct chunk filtering, i.e., it can be applied to a chunk of data rather than a + * table. This includes any filter that implements {@link ExposesChunkFilter} or {@link ConditionFilter} with + * exactly one column. + */ + @Override + public final boolean supportsChunkFiltering() { + return supportsChunkFiltering; + } + + /** + * Whether this filter supports filtering based on parquet metadata. + */ + @Override + public final boolean supportsMetadataFiltering() { + return isRangeFilter || isMatchFilter; + } + + @Override + public final boolean supportsInMemoryDataIndexFiltering() { + // Note: if there is a cheap way to check if the filter will never be applicable for an in-memory data index, we + // would like to add that check here. + return true; + } + + @Override + public final boolean supportsDeferredDataIndexFiltering() { + // Note: if there is a cheap way to check if the filter will never be applicable for a deferred data index, we + // would like to add that check here. + return true; + } + + /** + * The filter to use for parquet metadata filtering. Can only call when {@link #supportsMetadataFiltering()} is + * {@code true}. + */ + @Override + public final WhereFilter filterForMetadataFiltering() { + if (isRangeFilter) { + return ((RangeFilter) filter).getRealFilter(); + } + if (isMatchFilter) { + return filter; + } + throw new IllegalStateException("Should only use when supportsMetadataFiltering is true"); + } + + /** + * Get the behavior of this filter when applied to null values. This is lazily computed on first access. + */ + @Override + public FilterNullBehavior filterNullBehavior() { + FilterNullBehavior local = filterNullBehavior; + if (local == null) { + synchronized (this) { + local = filterNullBehavior; + if (local == null) { + local = computeFilterNullBehavior(); + filterNullBehavior = local; + } + } + } + return local; + } + + private FilterNullBehavior computeFilterNullBehavior() { + // Create a dummy table with a single row and column, and `null` entry, and apply the filter to see + // if the filter includes nulls. + final ColumnSource columnSource = columnSources.get(0); + final NullValueColumnSource nullValueColumnSource = + NullValueColumnSource.getInstance(columnSource.getType(), columnSource.getComponentType()); + final Map> columnSourceMap = + Map.of(filter.getColumns().get(0), nullValueColumnSource); + try (final SafeCloseable ignored = LivenessScopeStack.open()) { + final Table nullTestDummyTable = TableTools.newTable(1, columnSourceMap); + final TrackingRowSet rowSet = nullTestDummyTable.getRowSet(); + try (final RowSet result = filter.filter(rowSet, rowSet, nullTestDummyTable, false)) { + return result.isEmpty() ? FilterNullBehavior.EXCLUDES_NULLS : FilterNullBehavior.INCLUDES_NULLS; + } catch (final Exception e) { + return FilterNullBehavior.FAILS_ON_NULLS; + } + } + } + + /** + * Create a {@link UnifiedChunkFilter} for the {@link WhereFilter} that efficiently filters chunks of data. Every + * thread that uses this should create its own instance and must close it after use. Can only call when + * {@link #supportsChunkFiltering()} is {@code true} + * + * @param maxChunkSize the maximum size of the chunk that will be filtered + * @return the initialized {@link UnifiedChunkFilter} + */ + @Override + public final UnifiedChunkFilter createChunkFilter(final int maxChunkSize) { + if (!supportsChunkFiltering) { + throw new IllegalStateException("Filter does not support chunk filtering: " + Strings.of(filter)); + } + final Optional chunkFilter = ExposesChunkFilter.chunkFilter(filter); + if (chunkFilter.isPresent()) { + return new DirectChunkFilter(chunkFilter.get(), maxChunkSize); + } else if (filter instanceof ConditionFilter && ((ConditionFilter) filter).getNumInputsUsed() == 1) { + // Create a dummy table with no rows and single column of the correct type and name as the filter. This is + // used to extract a chunk filter kernel from the conditional filter and bind it to the correct name and + // type without capturing references to the actual table or its column sources. + final Table initTable = conditionalFilterInitTable(); + try { + final ConditionFilter conditionFilter = (ConditionFilter) filter; + final AbstractConditionFilter.Filter acfFilter = + conditionFilter.getFilter(initTable, initTable.getRowSet()); + return new ConditionKernelChunkFilter(acfFilter, maxChunkSize); + } catch (final Exception e) { + throw new IllegalArgumentException("Error creating condition filter in BasePushdownFilterContext", e); + } + } else { + throw new UnsupportedOperationException( + "Filter does not support chunk filtering: " + Strings.of(filter)); + } + } + + private Table conditionalFilterInitTable() { + Table local = conditionalFilterInitTable; + if (local == null) { + synchronized (this) { + local = conditionalFilterInitTable; + if (local == null) { + final Map> columnSourceMap = Map.of(filter.getColumns().get(0), + NullValueColumnSource.getInstance( + columnSources.get(0).getType(), + columnSources.get(0).getComponentType())); + local = TableTools.newTable(0, columnSourceMap); + conditionalFilterInitTable = local; + } + } + } + return local; + } + + /** + * A {@link UnifiedChunkFilter} that wraps a {@link ChunkFilter} directly. + */ + private static final class DirectChunkFilter implements UnifiedChunkFilter { + private final ChunkFilter chunkFilter; + private final WritableLongChunk resultChunk; + + private DirectChunkFilter(final ChunkFilter chunkFilter, final int maxChunkSize) { + this.chunkFilter = chunkFilter; + // We need to create a WritableLongChunk to hold the results of the chunk filter. + this.resultChunk = WritableLongChunk.makeWritableChunk(maxChunkSize); + } + + @Override + public LongChunk filter(Chunk values, LongChunk keys) { + chunkFilter.filter(values, keys, resultChunk); + return resultChunk; + } + + @Override + public void close() { + resultChunk.close(); + } + } + + /** + * A {@link UnifiedChunkFilter} that wraps a {@link ConditionFilter} by extracting its kernel and context. + */ + private static final class ConditionKernelChunkFilter implements UnifiedChunkFilter { + private final AbstractConditionFilter.Filter acfFilter; + private final ConditionFilter.FilterKernel.Context conditionFilterContext; + + private ConditionKernelChunkFilter( + final AbstractConditionFilter.Filter acfFilter, + final int maxChunkSize) { + this.acfFilter = acfFilter; + // Create the context for the ConditionFilter, which will be used to filter chunks. + this.conditionFilterContext = acfFilter.getContext(maxChunkSize); + } + + @Override + public LongChunk filter( + Chunk values, + LongChunk keys) { + // noinspection unchecked + return acfFilter.filter(conditionFilterContext, keys, new Chunk[] {values}); + } + + @Override + public void close() { + conditionFilterContext.close(); + } + } + + @MustBeInvokedByOverriders + @Override + public void close() { + conditionalFilterInitTable = null; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownFilterMatcher.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownFilterMatcher.java index 240799f671b..3d5299bb397 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownFilterMatcher.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownFilterMatcher.java @@ -43,14 +43,17 @@ public interface PushdownFilterMatcher { * that the filter cannot be pushed down. * @param onError Consumer of any exceptions that occur during the estimate operation */ - void estimatePushdownFilterCost( + default void estimatePushdownFilterCost( final WhereFilter filter, final RowSet selection, final boolean usePrev, final PushdownFilterContext context, final JobScheduler jobScheduler, final LongConsumer onComplete, - final Consumer onError); + final Consumer onError) { + // Default to having no benefit by pushing down. + onComplete.accept(Long.MAX_VALUE); + } /** * Push down the given filter to the underlying table and pass the result to the consumer. This method is expected @@ -76,7 +79,7 @@ void estimatePushdownFilterCost( * @param onComplete Consumer of the output rowsets for added and modified rows that pass the filter * @param onError Consumer of any exceptions that occur during the pushdown operation */ - void pushdownFilter( + default void pushdownFilter( final WhereFilter filter, final RowSet selection, final boolean usePrev, @@ -84,7 +87,10 @@ void pushdownFilter( final long costCeiling, final JobScheduler jobScheduler, final Consumer onComplete, - final Consumer onError); + final Consumer onError) { + // Default to returning all results as "maybe" + onComplete.accept(PushdownResult.allMaybeMatch(selection)); + } /** * Create a pushdown filter context for this entity. @@ -94,9 +100,11 @@ void pushdownFilter( * * @return the created filter context */ - PushdownFilterContext makePushdownFilterContext( + default PushdownFilterContext makePushdownFilterContext( final WhereFilter filter, - final List> filterSources); + final List> filterSources) { + return PushdownFilterContext.NO_PUSHDOWN_CONTEXT; + } /** * Given a filter and a list of column sources, return the appropriate {@link PushdownFilterMatcher} to use for diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownResult.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownResult.java index 9b6f538da81..18ff5de275f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownResult.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/PushdownResult.java @@ -28,6 +28,10 @@ public final class PushdownResult implements SafeCloseable { * The entire column contains a single value, so a single read is sufficient to determine matches. */ public static final long SINGLE_VALUE_COLUMN_COST = 1_000L; + /** + * The entire region contains a single value, so a single read is sufficient to determine matches. + */ + public static final long SINGLE_VALUE_REGION_COST = 2_000L; /** * Only table/row-group statistics are checked, assuming the metadata is already loaded */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexPushdownManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexPushdownManager.java index cb1564e1340..8a677a2b448 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexPushdownManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/dataindex/DataIndexPushdownManager.java @@ -153,7 +153,7 @@ public void pushdownFilter( onComplete.accept(PushdownResult.allMaybeMatch(selection)); } - public static class DataIndexPushdownContext extends BasePushdownFilterContext { + public static class DataIndexPushdownContext extends BasePushdownFilterContextImpl { private final Map renameMap; private final PushdownFilterContext wrappedContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java index 33b89ebd13c..f2d95907f2b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/TableLocation.java @@ -9,7 +9,7 @@ import io.deephaven.engine.liveness.LivenessReferent; import io.deephaven.engine.table.BasicDataIndex; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.PushdownFilterMatcher; +import io.deephaven.engine.table.impl.sources.regioned.RegionedPushdownFilterMatcher; import io.deephaven.io.log.impl.LogOutputStringImpl; import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.type.NamedImplementation; @@ -24,7 +24,8 @@ * source table. */ public interface TableLocation - extends NamedImplementation, LogOutputAppendable, TableLocationState, LivenessReferent, PushdownFilterMatcher { + extends NamedImplementation, LogOutputAppendable, TableLocationState, LivenessReferent, + RegionedPushdownFilterMatcher { /** * Listener interface for anything that wants to know about changes to a location. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java index 6fa8a282522..af7478c2b49 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/locations/impl/AbstractTableLocation.java @@ -10,6 +10,8 @@ import io.deephaven.engine.table.impl.PushdownFilterContext; import io.deephaven.engine.table.impl.PushdownResult; import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.regioned.RegionedPushdownAction; +import io.deephaven.engine.table.impl.sources.regioned.RegionedPushdownFilterContext; import io.deephaven.engine.table.impl.util.FieldUtils; import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.util.string.StringUtils; @@ -25,10 +27,12 @@ import java.lang.ref.SoftReference; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.stream.Collectors; /** * Partial TableLocation implementation for use by TableDataService implementations. @@ -300,6 +304,10 @@ public final BasicDataIndex getDataIndex(@NotNull final String... columns) { @Nullable public abstract BasicDataIndex loadDataIndex(@NotNull String... columns); + // ------------------------------------------------------------------------------------------------------------------ + // PushdownFilterMatcher default implementation + // ------------------------------------------------------------------------------------------------------------------ + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -309,8 +317,31 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Default to having no benefit by pushing down. - onComplete.accept(Long.MAX_VALUE); + if (selection.isEmpty()) { + // If the selection is empty, we can skip all pushdown filtering. + onComplete.accept(Long.MAX_VALUE); + return; + } + + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) context; + + // Generate a list of all the allowed actions, sorted by minimum cost. + final List sorted = supportedActions() + .stream() + .filter(action -> action.allows(this, filterCtx)) + .sorted(Comparator.comparingLong(RegionedPushdownAction::filterCost)) + .collect(Collectors.toList()); + + // If no modes are allowed, we can skip all pushdown filtering. + if (sorted.isEmpty()) { + onComplete.accept(Long.MAX_VALUE); + return; + } + + // Delegate to TableLocation to determine which of the supported actions applies to this particular location. + try (final RegionedPushdownAction.EstimateContext estimateCtx = makeEstimateContext(filter, filterCtx)) { + onComplete.accept(estimatePushdownAction(sorted, filter, selection, usePrev, filterCtx, estimateCtx)); + } } @Override @@ -323,8 +354,44 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Default to returning all results as "maybe" - onComplete.accept(PushdownResult.allMaybeMatch(selection)); + if (selection.isEmpty()) { + // If the selection is empty, we can skip all pushdown filtering. + onComplete.accept(PushdownResult.allMaybeMatch(selection)); + return; + } + + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) context; + + // Generate a list of all the supported allowed actions, sorted by minimum cost. + final List sorted = supportedActions() + .stream() + .filter(action -> action.allows(this, filterCtx, costCeiling)) + .sorted(Comparator.comparingLong(RegionedPushdownAction::filterCost)) + .collect(Collectors.toList()); + + // If no modes are allowed, we can skip all pushdown filtering. + if (sorted.isEmpty()) { + onComplete.accept(PushdownResult.allMaybeMatch(selection)); + return; + } + + // Initialize the pushdown result with the selection rowset as "maybe" rows + PushdownResult result = PushdownResult.allMaybeMatch(selection); + + // Delegate to TableLocation and perform each supported action.. + try (final RegionedPushdownAction.ActionContext actionCtx = makeActionContext(filter, filterCtx)) { + for (final RegionedPushdownAction action : sorted) { + try (final PushdownResult ignored = result) { + result = performPushdownAction(action, filter, selection, result, usePrev, filterCtx, actionCtx); + } + if (result.maybeMatch().isEmpty()) { + // No maybe rows remaining, so no reason to continue filtering. + break; + } + } + // Return the final result + onComplete.accept(result); + } } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java index 70a24f54a1f..b340a0a2734 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.select; +import io.deephaven.api.filter.Filter; import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; import java.util.Optional; @@ -14,4 +15,8 @@ public interface ExposesChunkFilter { * @return If available, returns the underlying chunk filter. Otherwise returns {@link Optional#empty()}. */ Optional chunkFilter(); + + static Optional chunkFilter(Filter filter) { + return (filter instanceof ExposesChunkFilter) ? ((ExposesChunkFilter) filter).chunkFilter() : Optional.empty(); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSingleValueSource.java index 01200d69ec4..7b255cd6407 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSingleValueSource.java @@ -12,21 +12,13 @@ import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_BOOLEAN; /** @@ -159,31 +151,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(get(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrev + return SingleValuePushdownHelper.makeChunk(getPrev(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java index 0f9177dbd59..9b7c92ce363 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.java @@ -54,14 +54,18 @@ public class BooleanSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForBoolean /* MIXIN_IMPLS */ { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY, - () -> new byte[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY2, - () -> new byte[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY1, - () -> new ByteOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY0, - () -> new ByteOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY, + () -> new byte[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY2, + () -> new byte[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY1, + () -> new ByteOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BOOLEAN_RECYCLER_CAPACITY0, + () -> new ByteOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSingleValueSource.java index 25eec4e8a7e..915af798b70 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_BYTE; import static io.deephaven.util.type.TypeUtils.unbox; @@ -170,31 +162,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getByte(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevByte + return SingleValuePushdownHelper.makeChunk(getPrevByte(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.java index 556cbf10658..5ecbc7011a3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.java @@ -48,14 +48,18 @@ public class ByteSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForByte /* MIXIN_IMPLS */ { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY, - () -> new byte[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY2, - () -> new byte[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY1, - () -> new ByteOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY0, - () -> new ByteOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY, + () -> new byte[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY2, + () -> new byte[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY1, + () -> new ByteOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.BYTE_RECYCLER_CAPACITY0, + () -> new ByteOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterSingleValueSource.java index 84253ce90df..b7278da2726 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/CharacterSingleValueSource.java @@ -6,22 +6,14 @@ import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_CHAR; import static io.deephaven.util.type.TypeUtils.unbox; @@ -166,31 +158,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getChar(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevChar + return SingleValuePushdownHelper.makeChunk(getPrevChar(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSingleValueSource.java index 846c746e22e..71503f47d24 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.DoubleChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_DOUBLE; import static io.deephaven.util.type.TypeUtils.unbox; @@ -170,31 +162,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getDouble(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevDouble + return SingleValuePushdownHelper.makeChunk(getPrevDouble(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.java index a46986975cb..c1b19341e82 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.java @@ -48,14 +48,18 @@ public class DoubleSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForDouble /* MIXIN_IMPLS */ { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY, - () -> new double[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY2, - () -> new double[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY1, - () -> new DoubleOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY0, - () -> new DoubleOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY, + () -> new double[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY2, + () -> new double[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY1, + () -> new DoubleOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.DOUBLE_RECYCLER_CAPACITY0, + () -> new DoubleOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSingleValueSource.java index ff2e4925fb8..a20ff94d424 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.FloatChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_FLOAT; import static io.deephaven.util.type.TypeUtils.unbox; @@ -170,31 +162,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getFloat(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevFloat + return SingleValuePushdownHelper.makeChunk(getPrevFloat(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.java index d350ab0762b..49c1dad2382 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.java @@ -48,14 +48,18 @@ public class FloatSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForFloat /* MIXIN_IMPLS */ { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY, - () -> new float[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY2, - () -> new float[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY1, - () -> new FloatOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY0, - () -> new FloatOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY, + () -> new float[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY2, + () -> new float[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY1, + () -> new FloatOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.FLOAT_RECYCLER_CAPACITY0, + () -> new FloatOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSingleValueSource.java index ed1190da65c..b8b40e70853 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_INT; import static io.deephaven.util.type.TypeUtils.unbox; @@ -170,31 +162,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getInt(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevInt + return SingleValuePushdownHelper.makeChunk(getPrevInt(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.java index 96a5a4c24b6..483e0a1c133 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.java @@ -48,14 +48,18 @@ public class IntegerSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForInt /* MIXIN_IMPLS */ { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY, - () -> new int[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY2, - () -> new int[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY1, - () -> new IntOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY0, - () -> new IntOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY, + () -> new int[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY2, + () -> new int[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY1, + () -> new IntOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.INT_RECYCLER_CAPACITY0, + () -> new IntOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSingleValueSource.java index 51691c7aeb5..841e012b168 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_LONG; import static io.deephaven.util.type.TypeUtils.unbox; @@ -170,31 +162,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getLong(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevLong + return SingleValuePushdownHelper.makeChunk(getPrevLong(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSparseArraySource.java index caa96092d49..8fa8c75e35e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/LongSparseArraySource.java @@ -56,14 +56,18 @@ public class LongSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForLong , ConvertibleTimeSource { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY, - () -> new long[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY2, - () -> new long[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY1, - () -> new LongOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY0, - () -> new LongOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY, + () -> new long[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY2, + () -> new long[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY1, + () -> new LongOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.LONG_RECYCLER_CAPACITY0, + () -> new LongOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NullValueColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NullValueColumnSource.java index fc99d30f122..136d5630b39 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NullValueColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/NullValueColumnSource.java @@ -16,6 +16,7 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.WritableColumnSource; import io.deephaven.engine.table.impl.AbstractColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; import io.deephaven.engine.table.impl.PushdownFilterContext; import io.deephaven.engine.table.impl.PushdownResult; import io.deephaven.engine.table.impl.select.WhereFilter; @@ -27,6 +28,7 @@ import org.jetbrains.annotations.Nullable; import java.util.LinkedHashMap; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; import java.util.stream.Collectors; @@ -318,6 +320,14 @@ public void fillFromChunkUnordered( // Assume all values in src are null, which will be true for any correct usage. } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + // Delegate to the shared code for SingleValuePushdownHelper + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -327,9 +337,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -342,8 +350,23 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS) { + onComplete.accept(PushdownResult.allMatch(selection)); + return; + } + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.EXCLUDES_NULLS) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + + onComplete.accept(PushdownResult.allMaybeMatch(selection)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectSingleValueSource.java index 6cb775c0a67..d7e59c09750 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ObjectSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.type.TypeUtils.unbox; /** @@ -158,31 +150,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(get(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrev + return SingleValuePushdownHelper.makeChunk(getPrev(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowKeyAgnosticChunkSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowKeyAgnosticChunkSource.java index b036b5e4f6c..dea2b951a5f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowKeyAgnosticChunkSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/RowKeyAgnosticChunkSource.java @@ -4,73 +4,10 @@ package io.deephaven.engine.table.impl.sources; import io.deephaven.chunk.attributes.Any; -import io.deephaven.engine.liveness.LivenessScopeStack; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.rowset.TrackingWritableRowSet; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; -import io.deephaven.util.SafeCloseable; - -import java.util.Map; -import java.util.function.Consumer; -import java.util.function.LongConsumer; /** * This is a marker interface for chunk sources that are agnostic of the row key when evaluating the value for a given * row key. */ public interface RowKeyAgnosticChunkSource extends FillUnordered { - static void estimatePushdownFilterCostHelper( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); - } - - static void pushdownFilterHelper( - final ColumnSource columnSource, - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - - if (selection.isEmpty()) { - onComplete.accept(PushdownResult.allNoMatch(selection)); - return; - } - - // Only need to test a single value from this column. Create a single row table, execute the filter, and return - // `selection` or the empty set depending on the result. - final String columnName = filter.getColumns().get(0); - try (final SafeCloseable ignored = LivenessScopeStack.open(); - final TrackingWritableRowSet rowSet = RowSetFactory.fromKeys(selection.firstRowKey()).toTracking()) { - - // Must create a dummy table since the original table is not available. - final Map> columnSourceMap = Map.of(columnName, columnSource); - final Table dummyTable = new QueryTable(rowSet, columnSourceMap); - try (final RowSet result = filter.filter(rowSet, rowSet, dummyTable, usePrev)) { - if (result.isEmpty()) { - // No rows match the filter, return empty selection - onComplete.accept(PushdownResult.allNoMatch(selection)); - } else { - // All rows match this filter, return the original selection as `match` rows. - onComplete.accept(PushdownResult.allMatch(selection)); - } - } - } - } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSingleValueSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSingleValueSource.java index 539f82fd213..170a33ea95d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSingleValueSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSingleValueSource.java @@ -10,22 +10,14 @@ import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.ShortChunk; import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.impl.PushdownFilterContext; -import io.deephaven.engine.table.impl.PushdownResult; -import io.deephaven.engine.table.impl.select.WhereFilter; -import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; -import java.util.function.Consumer; -import java.util.function.LongConsumer; - import static io.deephaven.util.QueryConstants.NULL_SHORT; import static io.deephaven.util.type.TypeUtils.unbox; @@ -170,31 +162,13 @@ public boolean providesFillUnordered() { } @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + public Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(getShort(0)); } @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + public Chunk getPrevValueChunk() { + // avoid duplicating the current vs prev logic in getPrevShort + return SingleValuePushdownHelper.makeChunk(getPrevShort(0)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.java index fccd8f2c5bc..41dc948f5fd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.java @@ -48,14 +48,18 @@ public class ShortSparseArraySource extends SparseArrayColumnSource implements MutableColumnSourceGetDefaults.ForShort /* MIXIN_IMPLS */ { // region recyclers - private static final SoftRecycler recycler = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY, - () -> new short[BLOCK_SIZE], null); - private static final SoftRecycler recycler2 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY2, - () -> new short[BLOCK2_SIZE][], null); - private static final SoftRecycler recycler1 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY1, - () -> new ShortOneOrN.Block2[BLOCK1_SIZE], null); - private static final SoftRecycler recycler0 = new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY0, - () -> new ShortOneOrN.Block1[BLOCK0_SIZE], null); + private static final SoftRecycler recycler = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY, + () -> new short[BLOCK_SIZE], null); + private static final SoftRecycler recycler2 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY2, + () -> new short[BLOCK2_SIZE][], null); + private static final SoftRecycler recycler1 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY1, + () -> new ShortOneOrN.Block2[BLOCK1_SIZE], null); + private static final SoftRecycler recycler0 = + new SoftRecycler<>(SparseArrayColumnSourceConfiguration.SHORT_RECYCLER_CAPACITY0, + () -> new ShortOneOrN.Block1[BLOCK0_SIZE], null); // endregion recyclers /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValueColumnSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValueColumnSource.java index 9ed913f46be..87bf7e8faf9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValueColumnSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValueColumnSource.java @@ -3,14 +3,22 @@ // package io.deephaven.engine.table.impl.sources; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSequence; -import io.deephaven.engine.table.ChunkSink; -import io.deephaven.engine.table.WritableColumnSource; -import io.deephaven.engine.table.impl.AbstractColumnSource; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import java.util.List; +import java.util.function.Consumer; +import java.util.function.LongConsumer; +import java.util.function.Supplier; + public abstract class SingleValueColumnSource extends AbstractColumnSource implements WritableColumnSource, ChunkSink, InMemoryColumnSource, RowKeyAgnosticChunkSource { @@ -122,4 +130,57 @@ public final void ensureCapacity(long capacity, boolean nullFilled) { public FillFromContext makeFillFromContext(int chunkCapacity) { return DEFAULT_FILL_FROM_INSTANCE; } + + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + + @Override + public void estimatePushdownFilterCost( + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext context, + final JobScheduler jobScheduler, + final LongConsumer onComplete, + final Consumer onError) { + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); + } + + @Override + public void pushdownFilter( + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext context, + final long costCeiling, + final JobScheduler jobScheduler, + final Consumer onComplete, + final Consumer onError) { + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = usePrev ? this::getPrevValueChunk : this::getValueChunk; + + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); + } + + /** + * Returns a chunk containing the value for this column source. + */ + protected abstract Chunk getValueChunk(); + + /** + * Returns a chunk containing the previous value for this columnSource. + */ + protected abstract Chunk getPrevValueChunk(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValuePushdownHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValuePushdownHelper.java new file mode 100644 index 00000000000..a4418519e96 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/SingleValuePushdownHelper.java @@ -0,0 +1,174 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.liveness.LivenessScopeStack; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.rowset.TrackingWritableRowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.BasePushdownFilterContextImpl; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.util.SafeCloseable; + +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; + +public class SingleValuePushdownHelper { + private static final long[] singleRowKeyArray = new long[] {0}; + private static final LongChunk singleRowKeyChunk = LongChunk.chunkWrap(singleRowKeyArray); + + public static LongChunk singleRowKeyChunk() { + return singleRowKeyChunk; + } + + /** + * Returns a new chunk containing the single byte value. + */ + public static Chunk makeChunk(byte value) { + final byte[] arr = new byte[] {value}; + return ByteChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single char value. + */ + public static Chunk makeChunk(char value) { + final char[] arr = new char[] {value}; + return CharChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single short value. + */ + public static Chunk makeChunk(short value) { + final short[] arr = new short[] {value}; + return ShortChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single int value. + */ + public static Chunk makeChunk(int value) { + final int[] arr = new int[] {value}; + return IntChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single long value. + */ + public static Chunk makeChunk(long value) { + final long[] arr = new long[] {value}; + return LongChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single float value. + */ + public static Chunk makeChunk(float value) { + final float[] arr = new float[] {value}; + return FloatChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single double value. + */ + public static Chunk makeChunk(double value) { + final double[] arr = new double[] {value}; + return DoubleChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single boolean value. + */ + public static Chunk makeChunk(boolean value) { + final boolean[] arr = new boolean[] {value}; + return BooleanChunk.chunkWrap(arr); + } + + /** + * Returns a new chunk containing the single object value. + */ + public static Chunk makeChunk(Object value) { + final Object[] arr = new Object[] {value}; + return ObjectChunk.chunkWrap(arr); + } + + /** + * A pushdown filter context for row key agnostic chunk sources. + */ + public static class FilterContext extends BasePushdownFilterContextImpl { + public FilterContext( + final WhereFilter filter, + final List> columnSources) { + super(filter, columnSources); + } + } + + /** + * Execute the {@link #chunkFilter(BasePushdownFilterContext, Supplier)} or + * {@link #tableFilter(WhereFilter, RowSet, boolean, ColumnSource)} to test whether the single value matches the + * supplied filter. + */ + public static boolean filter( + final RowSet selection, + final boolean usePrev, + final BasePushdownFilterContext context, + final Supplier> valueChunkSupplier, + final ColumnSource columnSource) { + + // Chunk filtering has lower overhead than creating a dummy table. + if (context.supportsChunkFiltering()) { + return chunkFilter(context, valueChunkSupplier); + } + + return tableFilter(context.filter(), selection, usePrev, columnSource); + } + + /** + * Execute the chunk filter from the context and return {@code true} if the filter matches any rows. + */ + public static boolean chunkFilter( + final BasePushdownFilterContext context, + final Supplier> valueChunkSupplier) { + + try (final BasePushdownFilterContext.UnifiedChunkFilter chunkFilter = context.createChunkFilter(1)) { + final LongChunk resultChunk = + chunkFilter.filter(valueChunkSupplier.get(), SingleValuePushdownHelper.singleRowKeyChunk()); + return resultChunk.size() > 0; + } + } + + /** + * Execute the filter against a dummy table and return {@code true} if the filter matches any rows. + */ + public static boolean tableFilter( + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final ColumnSource columnSource) { + // Create a single row table, execute the filter, and return `selection` or empty depending on the result. + final String columnName = filter.getColumns().get(0); + + try (final SafeCloseable ignored = LivenessScopeStack.open(); + final TrackingWritableRowSet rowSet = RowSetFactory.fromKeys(selection.firstRowKey()).toTracking()) { + + // Create a dummy table containing only this column source. + final Map> columnSourceMap = Map.of(columnName, columnSource); + final Table dummyTable = new QueryTable(rowSet, columnSourceMap); + + // Execute the filter on the dummy table. + try (final RowSet result = filter.filter(rowSet, rowSet, dummyTable, usePrev)) { + return !result.isEmpty(); + } + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java index e9a0de73617..92482fa299d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/UnionSourceManager.java @@ -901,7 +901,7 @@ public void pushdownFilter( onError); } - public static class UnionSourcePushdownFilterContext extends BasePushdownFilterContext { + public static class UnionSourcePushdownFilterContext extends BasePushdownFilterContextImpl { final UnionSourceManager manager; final WritableRowSet maybeMatch; final Map renameMap; @@ -967,16 +967,16 @@ public void initialize(final RowSet selection, final boolean usePrev) { ? manager.constituentTables.getPrev(slot) : manager.constituentTables.get(slot); - final List> filterSources = filter.getColumns().stream() + final List> filterSources = filter().getColumns().stream() .map(cn -> renameMap.getOrDefault(cn, cn)) .map(constituent::getColumnSource).collect(Collectors.toList()); final PushdownFilterMatcher matcher = - PushdownFilterMatcher.getPushdownFilterMatcher(filter, filterSources); + PushdownFilterMatcher.getPushdownFilterMatcher(filter(), filterSources); if (matcher != null) { matchers.add(matcher); - contexts.add(matcher.makePushdownFilterContext(filter, filterSources)); + contexts.add(matcher.makePushdownFilterContext(filter(), filterSources)); firstRowKeys.add(firstKey); lastRowKeys.add(lastKey); } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantByteSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantByteSource.java index 57e70ab6dc5..5a713630c7d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantByteSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantByteSource.java @@ -9,6 +9,7 @@ import io.deephaven.engine.table.ColumnSource; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableChunk; @@ -16,14 +17,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -99,6 +103,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -108,9 +119,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -123,9 +132,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getByte(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantCharSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantCharSource.java index 3923e0da962..3d3baad88a0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantCharSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantCharSource.java @@ -3,6 +3,7 @@ // package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableChunk; @@ -10,14 +11,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -93,6 +97,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -102,9 +113,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -117,9 +126,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getChar(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantDoubleSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantDoubleSource.java index 21f22cbd93e..3ea74c045a5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantDoubleSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantDoubleSource.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableDoubleChunk; import io.deephaven.chunk.WritableChunk; @@ -14,14 +15,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -97,6 +101,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -106,9 +117,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -121,9 +130,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getDouble(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantFloatSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantFloatSource.java index 58730a29804..1fc89e7d4c0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantFloatSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantFloatSource.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableFloatChunk; import io.deephaven.chunk.WritableChunk; @@ -14,14 +15,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -97,6 +101,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -106,9 +117,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -121,9 +130,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getFloat(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantInstantSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantInstantSource.java index 28c2799015d..db7839e28c2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantInstantSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantInstantSource.java @@ -3,8 +3,11 @@ // package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.time.DateTimeUtils; import org.jetbrains.annotations.NotNull; @@ -38,4 +41,9 @@ protected long toNanos(Instant value) { public ColumnSource toInstant() { return this; } + + @Override + protected Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(get(0)); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantIntSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantIntSource.java index c0d80f7561c..00ad04fd52d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantIntSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantIntSource.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableIntChunk; import io.deephaven.chunk.WritableChunk; @@ -14,14 +15,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -97,6 +101,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -106,9 +117,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -121,9 +130,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getInt(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantLongSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantLongSource.java index 6b34a8d7ebf..82350f4614e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantLongSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantLongSource.java @@ -16,6 +16,7 @@ import io.deephaven.engine.table.ColumnSource; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableLongChunk; import io.deephaven.chunk.WritableChunk; @@ -29,8 +30,10 @@ import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -106,6 +109,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -115,9 +125,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -130,9 +138,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getLong(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantNanosBasedTimeSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantNanosBasedTimeSource.java index a727d842a7d..77238c24ac4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantNanosBasedTimeSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantNanosBasedTimeSource.java @@ -4,6 +4,7 @@ package io.deephaven.engine.table.impl.sources.immutable; import io.deephaven.base.verify.Require; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableChunk; import io.deephaven.chunk.WritableObjectChunk; @@ -25,8 +26,10 @@ import java.time.LocalTime; import java.time.ZoneId; import java.time.ZonedDateTime; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; public abstract class ImmutableConstantNanosBasedTimeSource extends AbstractColumnSource implements InMemoryColumnSource, RowKeyAgnosticChunkSource, @@ -171,6 +174,18 @@ public ColumnSource toEpochNano() { } // endregion Reinterpretation + /** + * Returns a chunk containing the value for this column source. + */ + protected abstract Chunk getValueChunk(); + + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -180,9 +195,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -195,8 +208,14 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, this::getValueChunk, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantObjectSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantObjectSource.java index c8a7d9d7bcc..64cc188c455 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantObjectSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantObjectSource.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableObjectChunk; import io.deephaven.chunk.WritableChunk; @@ -14,14 +15,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -96,6 +100,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -105,9 +116,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -120,9 +129,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(get(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantShortSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantShortSource.java index 28b9841268a..16a4bd44c23 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantShortSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantShortSource.java @@ -7,6 +7,7 @@ // @formatter:off package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.WritableShortChunk; import io.deephaven.chunk.WritableChunk; @@ -14,14 +15,17 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.*; import io.deephaven.engine.table.impl.util.JobScheduler; import org.jetbrains.annotations.NotNull; +import java.util.List; import java.util.function.Consumer; import java.util.function.LongConsumer; +import java.util.function.Supplier; import static io.deephaven.engine.rowset.RowSequence.NULL_ROW_KEY; @@ -97,6 +101,13 @@ public boolean providesFillUnordered() { return true; } + @Override + public PushdownFilterContext makePushdownFilterContext( + final WhereFilter filter, + final List> filterSources) { + return new SingleValuePushdownHelper.FilterContext(filter, filterSources); + } + @Override public void estimatePushdownFilterCost( final WhereFilter filter, @@ -106,9 +117,7 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.estimatePushdownFilterCostHelper( - filter, selection, usePrev, context, jobScheduler, onComplete, onError); + onComplete.accept(PushdownResult.SINGLE_VALUE_COLUMN_COST); } @Override @@ -121,9 +130,16 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the shared code for RowKeyAgnosticChunkSource - RowKeyAgnosticChunkSource.pushdownFilterHelper(this, filter, selection, usePrev, context, costCeiling, - jobScheduler, onComplete, onError); + if (selection.isEmpty()) { + onComplete.accept(PushdownResult.allNoMatch(selection)); + return; + } + final SingleValuePushdownHelper.FilterContext filterCtx = (SingleValuePushdownHelper.FilterContext) context; + + final Supplier> chunkSupplier = () -> SingleValuePushdownHelper.makeChunk(getShort(0)); + final boolean matches = + SingleValuePushdownHelper.filter(selection, usePrev, filterCtx, chunkSupplier, this); + onComplete.accept(matches ? PushdownResult.allMatch(selection) : PushdownResult.allNoMatch(selection)); } // region reinterpretation diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantZonedDateTimeSource.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantZonedDateTimeSource.java index ab6407865e0..b042395e47e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantZonedDateTimeSource.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/immutable/ImmutableConstantZonedDateTimeSource.java @@ -3,9 +3,12 @@ // package io.deephaven.engine.table.impl.sources.immutable; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.ImmutableColumnSourceGetDefaults; import io.deephaven.engine.table.impl.sources.ConvertibleTimeSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.time.DateTimeUtils; import org.jetbrains.annotations.NotNull; @@ -49,4 +52,10 @@ public ColumnSource toZonedDateTime(@NotNull final ZoneId zone) { public ZoneId getZone() { return zone; } + + + @Override + protected Chunk getValueChunk() { + return SingleValuePushdownHelper.makeChunk(get(0)); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegion.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegion.java index 4301e3b76af..cd56f9bfff5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegion.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/ColumnRegion.java @@ -3,14 +3,23 @@ // package io.deephaven.engine.table.impl.sources.regioned; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.Releasable; import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.Page; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; -public interface ColumnRegion extends Page, Releasable { +import java.util.List; + +public interface ColumnRegion extends Page, Releasable, RegionedPushdownFilterMatcher { @Override @FinalDefault @@ -27,6 +36,15 @@ abstract class Null extends GenericColumnRegionBase implements ColumnRegion, WithDefaultsForRepeatingValues { + private static final RegionedPushdownAction.Region NULL_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Null); + private static final List SUPPORTED_ACTIONS = List.of(NULL_COLUMN_REGION); + Null(final long pageMask) { super(pageMask); } @@ -39,5 +57,45 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.fillWithNullValue(offset, length); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return NULL_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) filterContext; + + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + return nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } } 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 21930c87968..147576a9d10 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 @@ -49,7 +49,7 @@ default byte getByte(@NotNull final FillContext context, final long elementIndex * @return {@code destination}, to enable method chaining */ byte[] getBytes(long firstElementIndex, - @NotNull byte[] destination, + byte[] destination, int destinationOffset, int length); @@ -69,7 +69,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionByte DEFAULT_INSTANCE = new ColumnRegionByte.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -79,7 +79,7 @@ public byte getByte(final long elementIndex) { } @Override - public byte[] getBytes(final long firstElementIndex, @NotNull final byte[] destination, + public byte[] getBytes(final long firstElementIndex, final byte[] destination, final int destinationOffset, final int length) { Arrays.fill(destination, destinationOffset, destinationOffset + length, QueryConstants.NULL_BYTE); return destination; @@ -111,7 +111,7 @@ public void fillChunkAppend(@NotNull final FillContext context, } @Override - public byte[] getBytes(final long firstElementIndex, @NotNull final byte[] destination, + public byte[] getBytes(final long firstElementIndex, final byte[] destination, final int destinationOffset, final int length) { Arrays.fill(destination, destinationOffset, destinationOffset + length, value); return destination; @@ -144,7 +144,7 @@ public byte getByte(@NotNull final FillContext context, final long elementIndex) } @Override - public byte[] getBytes(final long firstElementIndex, @NotNull final byte[] destination, + public byte[] getBytes(final long firstElementIndex, final byte[] destination, final int destinationOffset, final int length) { return lookupRegion(firstElementIndex).getBytes(firstElementIndex, destination, destinationOffset, length); } 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 1e3c502c395..ca75c2b404f 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 @@ -7,10 +7,22 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; + /** * Column region interface for regions that support fetching primitive chars. */ @@ -52,7 +64,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionChar DEFAULT_INSTANCE = new ColumnRegionChar.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -66,6 +78,15 @@ final class Constant extends GenericColumnRegionBase implements ColumnRegionChar, WithDefaultsForRepeatingValues { + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); + private final char value; public Constant(final long pageMask, final char value) { @@ -85,6 +106,60 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.asWritableCharChunk().fillWithValue(offset, length, value); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (value == QueryConstants.NULL_CHAR) { + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + final ColumnSource columnSource = + InMemoryColumnSource.makeImmutableConstantSource(char.class, null, value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 40082ae145a..38547d24ea8 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,10 +11,22 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; + /** * Column region interface for regions that support fetching primitive doubles. */ @@ -56,7 +68,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionDouble DEFAULT_INSTANCE = new ColumnRegionDouble.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -70,6 +82,15 @@ final class Constant extends GenericColumnRegionBase implements ColumnRegionDouble, WithDefaultsForRepeatingValues { + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); + private final double value; public Constant(final long pageMask, final double value) { @@ -89,6 +110,60 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.asWritableDoubleChunk().fillWithValue(offset, length, value); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (value == QueryConstants.NULL_DOUBLE) { + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + final ColumnSource columnSource = + InMemoryColumnSource.makeImmutableConstantSource(double.class, null, value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 d29be986b38..cc24f91759f 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,10 +11,22 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; + /** * Column region interface for regions that support fetching primitive floats. */ @@ -56,7 +68,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionFloat DEFAULT_INSTANCE = new ColumnRegionFloat.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -70,6 +82,15 @@ final class Constant extends GenericColumnRegionBase implements ColumnRegionFloat, WithDefaultsForRepeatingValues { + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); + private final float value; public Constant(final long pageMask, final float value) { @@ -89,6 +110,60 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.asWritableFloatChunk().fillWithValue(offset, length, value); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (value == QueryConstants.NULL_FLOAT) { + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + final ColumnSource columnSource = + InMemoryColumnSource.makeImmutableConstantSource(float.class, null, value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 e9149156f28..1b559d9f5ba 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,10 +11,22 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; + /** * Column region interface for regions that support fetching primitive ints. */ @@ -56,7 +68,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionInt DEFAULT_INSTANCE = new ColumnRegionInt.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -70,6 +82,15 @@ final class Constant extends GenericColumnRegionBase implements ColumnRegionInt, WithDefaultsForRepeatingValues { + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); + private final int value; public Constant(final long pageMask, final int value) { @@ -89,6 +110,60 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.asWritableIntChunk().fillWithValue(offset, length, value); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (value == QueryConstants.NULL_INT) { + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + final ColumnSource columnSource = + InMemoryColumnSource.makeImmutableConstantSource(int.class, null, value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 9bd16a1b759..0b664b72b49 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,10 +11,22 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; + /** * Column region interface for regions that support fetching primitive longs. */ @@ -56,7 +68,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionLong DEFAULT_INSTANCE = new ColumnRegionLong.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -70,6 +82,15 @@ final class Constant extends GenericColumnRegionBase implements ColumnRegionLong, WithDefaultsForRepeatingValues { + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); + private final long value; public Constant(final long pageMask, final long value) { @@ -89,6 +110,60 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.asWritableLongChunk().fillWithValue(offset, length, value); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (value == QueryConstants.NULL_LONG) { + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + final ColumnSource columnSource = + InMemoryColumnSource.makeImmutableConstantSource(long.class, null, value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 1af815ceb0e..78146bc00aa 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 @@ -5,6 +5,11 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; import io.deephaven.engine.table.impl.chunkattributes.DictionaryKeys; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; @@ -13,9 +18,15 @@ import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetBuilderSequential; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; +import java.util.Objects; import java.util.stream.IntStream; import static io.deephaven.util.QueryConstants.NULL_LONG; @@ -144,7 +155,7 @@ final class Null extends ColumnRegion.Null private ColumnRegionLong dictionaryKeysRegion; - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -181,6 +192,17 @@ final class Constant private static final ColumnRegionLong DEFAULT_SINGLETON_DICTIONARY_KEYS_REGION = new ColumnRegionLong.Constant<>(RegionedColumnSourceBase.PARAMETERS.regionMask, 0L); + /** + * The supported pushdown action for constant object regions. + */ + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); private final DATA_TYPE value; @@ -221,6 +243,63 @@ public ColumnRegionLong getDictionaryKeysRegion() { return dictionaryKeysRegion == null ? dictionaryKeysRegion = createConstantDictionaryKeysRegion(mask()) : dictionaryKeysRegion; } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (Objects.isNull(value)) { + // We can resolve all the maybe rows with one test. + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + @SuppressWarnings("unchecked") + final ColumnSource columnSource = InMemoryColumnSource.makeImmutableConstantSource( + (Class) value.getClass(), + value.getClass().getComponentType(), + value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 13f6d5d7989..a80718a257e 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,10 +11,22 @@ import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.page.PagingContextHolder; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetFactory; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.sources.InMemoryColumnSource; +import io.deephaven.engine.table.impl.sources.SingleValuePushdownHelper; import io.deephaven.util.QueryConstants; import io.deephaven.util.annotations.FinalDefault; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; +import java.util.List; + /** * Column region interface for regions that support fetching primitive shorts. */ @@ -56,7 +68,7 @@ final class Null extends ColumnRegion.Null implements Co private static final ColumnRegionShort DEFAULT_INSTANCE = new ColumnRegionShort.Null(RegionedColumnSourceBase.PARAMETERS.regionMask); - private Null(final long pageMask) { + public Null(final long pageMask) { super(pageMask); } @@ -70,6 +82,15 @@ final class Constant extends GenericColumnRegionBase implements ColumnRegionShort, WithDefaultsForRepeatingValues { + final static RegionedPushdownAction.Region CONSTANT_COLUMN_REGION = + new RegionedPushdownAction.Region( + () -> false, + PushdownResult.SINGLE_VALUE_REGION_COST, + (ctx) -> true, + (tl) -> true, + (cr) -> cr instanceof Constant); + private static final List SUPPORTED_ACTIONS = List.of(CONSTANT_COLUMN_REGION); + private final short value; public Constant(final long pageMask, final short value) { @@ -89,6 +110,60 @@ public void fillChunkAppend(@NotNull final FillContext context, destination.asWritableShortChunk().fillWithValue(offset, length, value); destination.setSize(offset + length); } + + @Override + public List supportedActions() { + return SUPPORTED_ACTIONS; + } + + @Override + @MustBeInvokedByOverriders + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return CONSTANT_COLUMN_REGION.filterCost(); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final BasePushdownFilterContext filterCtx = (BasePushdownFilterContext) filterContext; + + final boolean matches; + if (value == QueryConstants.NULL_SHORT) { + final BasePushdownFilterContext.FilterNullBehavior nullBehavior = filterCtx.filterNullBehavior(); + if (nullBehavior == BasePushdownFilterContext.FilterNullBehavior.FAILS_ON_NULLS) { + // Bad-behaving filter, but not our responsibility to handle during pushdown. + return input.copy(); + } + matches = nullBehavior == BasePushdownFilterContext.FilterNullBehavior.INCLUDES_NULLS; + } else { + if (filterCtx.supportsChunkFiltering()) { + matches = SingleValuePushdownHelper.chunkFilter(filterCtx, + () -> SingleValuePushdownHelper.makeChunk(value)); + } else { + final ColumnSource columnSource = + InMemoryColumnSource.makeImmutableConstantSource(short.class, null, value); + matches = SingleValuePushdownHelper.tableFilter(filter, selection, false, columnSource); + } + } + return matches + // Promote all maybe rows to match. + ? PushdownResult.of(selection, input.match().union(input.maybeMatch()), RowSetFactory.empty()) + // None of these rows match, return the original match rows. + : PushdownResult.of(selection, input.match(), RowSetFactory.empty()); + } } final class StaticPageStore 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 be4a91b246b..28ace52d075 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 @@ -7,9 +7,15 @@ import io.deephaven.chunk.attributes.Any; import io.deephaven.chunk.*; import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; +import org.jetbrains.annotations.MustBeInvokedByOverriders; import org.jetbrains.annotations.NotNull; import javax.annotation.OverridingMethodsMustInvokeSuper; +import java.util.List; import java.util.function.Supplier; /** @@ -92,4 +98,35 @@ 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 List supportedActions() { + return getResultRegion().supportedActions(); + } + + @Override + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return getResultRegion().estimatePushdownAction(actions, filter, selection, usePrev, filterContext, + estimateContext); + } + + @Override + @MustBeInvokedByOverriders + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + return getResultRegion().performPushdownAction(action, filter, selection, input, usePrev, filterContext, + actionContext); + } } 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 4a2732092ec..847ef304a0d 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 @@ -3,8 +3,23 @@ // package io.deephaven.engine.table.impl.sources.regioned; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Any; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownResult; import io.deephaven.engine.table.impl.locations.InvalidatedRegionException; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.select.WhereFilter; +import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.util.SafeCloseable; + +import java.util.Comparator; +import java.util.List; +import java.util.function.Consumer; +import java.util.function.LongConsumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Base {@link ColumnRegion} implementation. @@ -33,4 +48,139 @@ protected final void throwIfInvalidated() { throw new InvalidatedRegionException("Column region has been invalidated due to data removal"); } } + + @Override + public void estimatePushdownFilterCost( + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext context, + final JobScheduler jobScheduler, + final LongConsumer onComplete, + final Consumer onError) { + if (selection.isEmpty()) { + // If the selection is empty, we can skip all pushdown filtering. + onComplete.accept(Long.MAX_VALUE); + return; + } + + final RegionedPushdownFilterLocationContext filterCtx = (RegionedPushdownFilterLocationContext) context; + Assert.neqNull(filterCtx.tableLocation(), "filterCtx.tableLocation()"); + + final TableLocation tableLocation = filterCtx.tableLocation(); + + // We must consider all the actions from this region and from the AbstractTableLocation and execute them in + // minimal cost order. + final List sortedRegion = supportedActions() + .stream() + .filter(action -> ((RegionedPushdownAction.Region) action).allows(tableLocation, this, filterCtx)) + .sorted(Comparator.comparingLong(RegionedPushdownAction::filterCost)) + .collect(Collectors.toList()); + + final long regionCost; + if (!sortedRegion.isEmpty()) { + try (final RegionedPushdownAction.EstimateContext estimateCtx = makeEstimateContext(filter, filterCtx)) { + regionCost = estimatePushdownAction(sortedRegion, filter, selection, usePrev, filterCtx, estimateCtx); + } + } else { + regionCost = Long.MAX_VALUE; + } + + // Consider the location actions that are less expensive than the lowest cost region action. + final List sortedLocation = tableLocation.supportedActions() + .stream() + .filter(action -> action.allows(tableLocation, filterCtx)) + .filter(action -> action.filterCost() < regionCost) + .sorted(Comparator.comparingLong(RegionedPushdownAction::filterCost)) + .collect(Collectors.toList()); + + final long locationCost; + if (!sortedLocation.isEmpty()) { + try (final RegionedPushdownAction.EstimateContext estimateCtx = + tableLocation.makeEstimateContext(filter, filterCtx)) { + locationCost = + tableLocation.estimatePushdownAction(sortedLocation, filter, selection, usePrev, filterCtx, + estimateCtx); + } + } else { + locationCost = Long.MAX_VALUE; + } + + // Return the lowest cost operation. + onComplete.accept(Math.min(regionCost, locationCost)); + } + + @Override + public void pushdownFilter( + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext context, + final long costCeiling, + final JobScheduler jobScheduler, + final Consumer onComplete, + final Consumer onError) { + if (selection.isEmpty()) { + // If the selection is empty, we can skip all pushdown filtering. + onComplete.accept(PushdownResult.allMaybeMatch(selection)); + return; + } + + final RegionedPushdownFilterLocationContext filterCtx = (RegionedPushdownFilterLocationContext) context; + Assert.neqNull(filterCtx.tableLocation(), "filterCtx.tableLocation()"); + + final TableLocation tableLocation = filterCtx.tableLocation(); + + // We must consider all the actions from this region and from the AbstractTableLocation and execute them in + // minimal cost order + final List sorted = + Stream.concat(supportedActions().stream(), tableLocation.supportedActions().stream()) + .filter(action -> action instanceof RegionedPushdownAction.Region + ? ((RegionedPushdownAction.Region) action).allows(tableLocation, this, filterCtx, + costCeiling) + : action.allows(tableLocation, filterCtx, costCeiling)) + .sorted(Comparator.comparingLong(RegionedPushdownAction::filterCost)) + .collect(Collectors.toList()); + + // If no modes are allowed, we can skip all pushdown filtering. + if (sorted.isEmpty()) { + onComplete.accept(PushdownResult.allMaybeMatch(selection)); + return; + } + + // Initialize the pushdown result with the selection rowset as "maybe" rows + PushdownResult result = PushdownResult.allMaybeMatch(selection); + + // Deferred contexts, to be created only when needed. + RegionedPushdownAction.ActionContext regionCtx = null; + RegionedPushdownAction.ActionContext locationCtx = null; + + // Iterate through the sorted actions + for (final RegionedPushdownAction action : sorted) { + try (final PushdownResult ignored = result) { + if (action instanceof RegionedPushdownAction.Location) { + result = tableLocation.performPushdownAction(action, filter, selection, result, usePrev, filterCtx, + locationCtx == null + ? (locationCtx = tableLocation.makeActionContext(filter, filterCtx)) + : locationCtx); + } else { + result = performPushdownAction(action, filter, selection, result, usePrev, filterCtx, + regionCtx == null + ? (regionCtx = makeActionContext(filter, filterCtx)) + : regionCtx); + + } + } + if (result.maybeMatch().isEmpty()) { + // No maybe rows remaining, so no reason to continue filtering. + break; + } + } + + // noinspection ConstantConditions + SafeCloseable.closeAll(regionCtx, locationCtx); + + // Return the final result + onComplete.accept(result); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java index 1bc9ee6748f..4902dbeb009 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceBase.java @@ -3,20 +3,25 @@ // package io.deephaven.engine.table.impl.sources.regioned; +import io.deephaven.base.log.LogOutput; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.*; import io.deephaven.chunk.WritableChunk; import io.deephaven.engine.rowset.RowSequence; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.util.JobScheduler; +import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.annotations.TestUseOnly; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.LongConsumer; @@ -114,9 +119,53 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - // Delegate to the manager. - manager.estimatePushdownFilterCost(filter, selection, usePrev, context, jobScheduler, - onComplete, onError); + final RegionedPushdownFilterContext filterContext = (RegionedPushdownFilterContext) context; + final List tleList = + manager.includedLocationEntries(); + + // Sample a few regions and return the lowest cost + final int[] regionIndices = RegionedColumnSourceManager.regionIndices(selection, + RegionedColumnSourceManager.PUSHDOWN_LOCATION_SAMPLES); + + final AtomicLong minCost = new AtomicLong(Long.MAX_VALUE); + + jobScheduler.iterateParallel( + ExecutionContext.getContext(), + (LogOutput output) -> output.append("RegionedColumnSourceBase#estimatePushdownFilterCost"), + RegionedPushdownHelper.RegionThreadContext::new, + 0, + regionIndices.length, + (ctx, idx, nec, resume) -> { + ctx.reset(); + final int regionIndex = regionIndices[idx]; + + // Create a local pushdown context that can provide the table location to the ColumnRegion + final RegionedColumnSourceManager.IncludedTableLocationEntry tle = tleList.get(regionIndex); + final RegionedPushdownFilterLocationContext newCtx = filterContext.withTableLocation(tle.location); + try { + ctx.shiftedRowSet = tle.subsetAndShiftIntoLocationSpace(selection); + getRegion(regionIndex).estimatePushdownFilterCost( + filter, + ctx.shiftedRowSet, + usePrev, + newCtx, + jobScheduler, + regionCost -> { + minCost.updateAndGet(old -> Math.min(old, regionCost)); + resume.run(); + newCtx.close(); + }, + nec); + } catch (final Exception e) { + // In the case of an exception, clean up the temporary context. + newCtx.close(); + throw e; + } + }, + () -> onComplete.accept(minCost.get()), + () -> { + }, + onError); } @Override @@ -129,9 +178,61 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - // Delegate to the manager. - manager.pushdownFilter(filter, selection, usePrev, context, costCeiling, jobScheduler, - onComplete, onError); + final RegionedPushdownFilterContext filterContext = (RegionedPushdownFilterContext) context; + + final List tleList = + manager.includedLocationEntries(); + + final int[] regionIndices = RegionedColumnSourceManager.regionIndices(selection, Integer.MAX_VALUE); + + final WritableRowSet[] matches = new WritableRowSet[regionIndices.length]; + final WritableRowSet[] maybeMatches = new WritableRowSet[regionIndices.length]; + + jobScheduler.iterateParallel( + ExecutionContext.getContext(), + (LogOutput output) -> output.append("RegionedColumnSourceBase#pushdownFilter"), + RegionedPushdownHelper.RegionThreadContext::new, + 0, + regionIndices.length, + (ctx, idx, nec, resume) -> { + ctx.reset(); + final int regionIndex = regionIndices[idx]; + + final RegionedColumnSourceManager.IncludedTableLocationEntry tle = tleList.get(regionIndex); + // Create a local pushdown context that can provide the table location to the ColumnRegion + final RegionedPushdownFilterLocationContext newCtx = + filterContext.withTableLocation(tle.location); + + ctx.shiftedRowSet = tle.subsetAndShiftIntoLocationSpace(selection); + + try { + getRegion(regionIndex).pushdownFilter( + filter, + ctx.shiftedRowSet, + usePrev, + newCtx, + costCeiling, + jobScheduler, + result -> { + tle.unshiftIntoRegionSpace(result); + matches[idx] = result.match(); + maybeMatches[idx] = result.maybeMatch(); + resume.run(); + newCtx.close(); + }, + nec); + } catch (final Exception e) { + // In the case of an exception, clean up the temporary context. + newCtx.close(); + throw e; + } + }, + () -> onComplete.accept(RegionedPushdownHelper.buildResults(matches, maybeMatches, selection)), + () -> { + SafeCloseableArray.close(matches); + SafeCloseableArray.close(maybeMatches); + }, + onError); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceByte.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceByte.java index d3709bf0fe5..21026afe18c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceByte.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceByte.java @@ -90,7 +90,9 @@ public ColumnRegionByte makeRegion(@NotNull final ColumnDefinition co "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Byte at location " + locationKey); } - return new ColumnRegionByte.Constant<>(regionMask(), unbox((Byte) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionByte.createNull(regionMask()) + : new ColumnRegionByte.Constant<>(regionMask(), unbox((Byte) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceChar.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceChar.java index f90429ac4ce..08e1a32218c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceChar.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceChar.java @@ -74,7 +74,9 @@ public ColumnRegionChar makeRegion(@NotNull final ColumnDefinition co "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Character at location " + locationKey); } - return new ColumnRegionChar.Constant<>(regionMask(), unbox((Character) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionChar.createNull(regionMask()) + : new ColumnRegionChar.Constant<>(regionMask(), unbox((Character) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceDouble.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceDouble.java index ca166063263..4d6396ec9bd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceDouble.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceDouble.java @@ -78,7 +78,9 @@ public ColumnRegionDouble makeRegion(@NotNull final ColumnDefinition "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Double at location " + locationKey); } - return new ColumnRegionDouble.Constant<>(regionMask(), unbox((Double) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionDouble.createNull(regionMask()) + : new ColumnRegionDouble.Constant<>(regionMask(), unbox((Double) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceFloat.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceFloat.java index d0692f923ef..3a3d83a9914 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceFloat.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceFloat.java @@ -78,7 +78,9 @@ public ColumnRegionFloat makeRegion(@NotNull final ColumnDefinition c "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Float at location " + locationKey); } - return new ColumnRegionFloat.Constant<>(regionMask(), unbox((Float) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionFloat.createNull(regionMask()) + : new ColumnRegionFloat.Constant<>(regionMask(), unbox((Float) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceInt.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceInt.java index c26fe34bac6..4da0d6f363d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceInt.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceInt.java @@ -78,7 +78,9 @@ public ColumnRegionInt makeRegion(@NotNull final ColumnDefinition col "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Integer at location " + locationKey); } - return new ColumnRegionInt.Constant<>(regionMask(), unbox((Integer) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionInt.createNull(regionMask()) + : new ColumnRegionInt.Constant<>(regionMask(), unbox((Integer) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceLong.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceLong.java index 7b175cba870..e25da13160d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceLong.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceLong.java @@ -138,7 +138,9 @@ public ColumnRegionLong makeRegion(@NotNull final ColumnDefinition co "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Long at location " + locationKey); } - return new ColumnRegionLong.Constant<>(regionMask(), unbox((Long) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionLong.createNull(regionMask()) + : new ColumnRegionLong.Constant<>(regionMask(), unbox((Long) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java index 7c46a912210..294c1f9b8cb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceManager.java @@ -27,12 +27,13 @@ import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; import io.deephaven.util.SafeCloseable; +import io.deephaven.util.SafeCloseableArray; import io.deephaven.util.annotations.ReferentialIntegrity; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import java.util.*; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.LongConsumer; import java.util.stream.Collectors; @@ -54,7 +55,7 @@ public class RegionedColumnSourceManager * How many locations to test for data index or other location-level metadata before we give up and assume the * location has no useful information for push-down purposes. */ - private static final int PUSHDOWN_LOCATION_SAMPLES = Configuration.getInstance() + static final int PUSHDOWN_LOCATION_SAMPLES = Configuration.getInstance() .getIntegerForClassWithDefault(RegionedColumnSourceManager.class, "pushdownLocationSamples", 5); /** @@ -75,7 +76,7 @@ public class RegionedColumnSourceManager /** * The column definitions of this table as a map from column name. */ - private final Map> columnNameToDefinition; + private volatile Map> columnNameToDefinition; /** * The column sources that make up this table. @@ -186,13 +187,11 @@ public class RegionedColumnSourceManager this.isRefreshing = isRefreshing; this.columnDefinitions = columnDefinitions; - this.columnNameToDefinition = new HashMap<>(columnDefinitions.size()); for (final ColumnDefinition columnDefinition : columnDefinitions) { final String columnName = columnDefinition.getName(); columnSources.put( columnName, componentFactory.createRegionedColumnSource(this, columnDefinition, codecMappings)); - columnNameToDefinition.put(columnName, columnDefinition); } // Create the table that will hold the location data @@ -552,7 +551,7 @@ public final synchronized Collection includedLocations() { .collect(Collectors.toCollection(ArrayList::new)); } - private synchronized ArrayList includedLocationEntries() { + synchronized ArrayList includedLocationEntries() { return new ArrayList<>(orderedIncludedTableLocations); } @@ -637,9 +636,9 @@ public ImmutableTableLocationKey getKey( /** * State-keeper for a table location and its column locations, once it's been found to have a positive size. */ - private class IncludedTableLocationEntry implements Comparable { + class IncludedTableLocationEntry implements Comparable { - private final TableLocation location; + final TableLocation location; private final TableLocationUpdateSubscriptionBuffer subscriptionBuffer; // New regions indices are assigned in order of insertion, starting from 0 with no re-use of removed indices. @@ -761,7 +760,7 @@ public int compareTo(@NotNull final IncludedTableLocationEntry other) { return Integer.compare(regionIndex, other.regionIndex); } - private WritableRowSet subsetAndShiftIntoLocationSpace(final RowSet selection) { + WritableRowSet subsetAndShiftIntoLocationSpace(final RowSet selection) { final long locationStartKey = firstRowKey(); // Extract the portion of selection that overlaps this region. final WritableRowSet overlappingRows = selection.subSetByKeyRange(locationStartKey, lastRowKey()); @@ -823,7 +822,7 @@ public Map getTableAttributes( return attributes; } - private static int[] regionIndices(final RowSet selection, final int maxCount) { + static int[] regionIndices(final RowSet selection, final int maxCount) { try (final RegionIndexIterator rit = RegionIndexIterator.of(selection)) { final IntStream.Builder builder = IntStream.builder(); for (int i = 0; i < maxCount && rit.hasNext(); ++i) { @@ -842,9 +841,57 @@ public void estimatePushdownFilterCost( final JobScheduler jobScheduler, final LongConsumer onComplete, final Consumer onError) { - final int[] regionIndices = regionIndices(selection, PUSHDOWN_LOCATION_SAMPLES); - new EstimateJobBuilder(selection.copy(), regionIndices, onComplete, onError) - .iterateParallel(jobScheduler, filter, usePrev, context); + // Sample a few regions and return the lowest cost + final int[] regionIndices = RegionedColumnSourceManager.regionIndices(selection, + RegionedColumnSourceManager.PUSHDOWN_LOCATION_SAMPLES); + + final RowSet localSelection = selection.copy(); + final AtomicLong minCost = new AtomicLong(Long.MAX_VALUE); + + final class RegionContext implements JobScheduler.JobThreadContext { + RowSet shiftedRowSet; + + @Override + public void close() { + SafeCloseable.closeAll(shiftedRowSet); + } + + public void reset() { + if (shiftedRowSet != null) { + shiftedRowSet.close(); + shiftedRowSet = null; + } + } + } + + jobScheduler.iterateParallel( + ExecutionContext.getContext(), + (LogOutput output) -> output.append("RegionedColumnSourceManager#estimatePushdownFilterCost"), + RegionContext::new, + 0, + regionIndices.length, + (ctx, idx, nec, resume) -> { + ctx.reset(); + final int regionIndex = regionIndices[idx]; + + final IncludedTableLocationEntry tle = orderedIncludedTableLocations.get(regionIndex); + ctx.shiftedRowSet = tle.subsetAndShiftIntoLocationSpace(localSelection); + + tle.location.estimatePushdownFilterCost( + filter, + ctx.shiftedRowSet, + usePrev, + context, + jobScheduler, + cost -> { + minCost.updateAndGet(old -> Math.min(old, cost)); + resume.run(); + }, + nec); + }, + () -> onComplete.accept(minCost.get()), + localSelection::close, + onError); } @Override @@ -857,390 +904,118 @@ public void pushdownFilter( final JobScheduler jobScheduler, final Consumer onComplete, final Consumer onError) { - final int[] regionIndices = regionIndices(selection, Integer.MAX_VALUE); - new PushdownJobBuilder(selection.copy(), regionIndices, onComplete, onError) - .iterateParallel(jobScheduler, filter, usePrev, context, costCeiling); + final int[] regionIndices = RegionedColumnSourceManager.regionIndices(selection, Integer.MAX_VALUE); + + final WritableRowSet[] matches = new WritableRowSet[regionIndices.length]; + final WritableRowSet[] maybeMatches = new WritableRowSet[regionIndices.length]; + + jobScheduler.iterateParallel( + ExecutionContext.getContext(), + (LogOutput output) -> output.append("RegionedColumnSourceManager#pushdownFilter"), + RegionedPushdownHelper.RegionThreadContext::new, + 0, + regionIndices.length, + (ctx, idx, nec, resume) -> { + ctx.reset(); + final int regionIndex = regionIndices[idx]; + + final IncludedTableLocationEntry tle = orderedIncludedTableLocations.get(regionIndex); + ctx.shiftedRowSet = tle.subsetAndShiftIntoLocationSpace(selection); + + tle.location.pushdownFilter( + filter, + ctx.shiftedRowSet, + usePrev, + context, + costCeiling, + jobScheduler, + result -> { + tle.unshiftIntoRegionSpace(result); + matches[idx] = result.match(); + maybeMatches[idx] = result.maybeMatch(); + resume.run(); + }, + nec); + }, + () -> onComplete.accept(RegionedPushdownHelper.buildResults(matches, maybeMatches, selection)), + () -> { + SafeCloseableArray.close(matches); + SafeCloseableArray.close(maybeMatches); + }, + onError); } /** * Get (or create) a map from column source to column name. */ private IdentityHashMap, String> columnSourceToName() { - if (columnSourceToName == null) { + IdentityHashMap, String> local = columnSourceToName; + if (local == null) { synchronized (this) { - if (columnSourceToName == null) { + local = columnSourceToName; + if (local == null) { final IdentityHashMap, String> tmp = new IdentityHashMap<>(columnSources.size()); columnSources.forEach((name, src) -> tmp.put(src, name)); - columnSourceToName = tmp; + columnSourceToName = local = tmp; } } } - return columnSourceToName; + return local; } - public static class RegionedColumnSourcePushdownFilterContext extends BasePushdownFilterContext { - private final List> columnDefinitions; - private final Map renameMap; - - public RegionedColumnSourcePushdownFilterContext( - final RegionedColumnSourceManager manager, - final WhereFilter filter, - final List> columnSources) { - super(filter, columnSources); - - final List filterColumns = filter.getColumns(); - Require.eq(filterColumns.size(), "filterColumns.size()", - columnSources.size(), "columnSources.size()"); - - // Map the incoming column sources to their local name and definition. - columnDefinitions = new ArrayList<>(columnSources.size()); - renameMap = new HashMap<>(); - final IdentityHashMap, String> columnSourceToName = manager.columnSourceToName(); - final Map> columnNameToDefinition = manager.columnNameToDefinition; - - for (int ii = 0; ii < filterColumns.size(); ii++) { - final String filterColumnName = filterColumns.get(ii); - final ColumnSource filterSource = columnSources.get(ii); - final String localColumnName = columnSourceToName.get(filterSource); - if (localColumnName == null) { - throw new IllegalArgumentException( - "No associated source for '" + filterColumnName + "' found in column sources"); - } - // Add the definition. - columnDefinitions.add(columnNameToDefinition.get(localColumnName)); - - // Add the rename (if needed) - if (localColumnName.equals(filterColumnName)) { - continue; + /** + * Get (or create) a map from column name to column definition. + */ + private Map> columnNameToDefinition() { + Map> local = columnNameToDefinition; + if (local == null) { + synchronized (this) { + local = columnNameToDefinition; + if (local == null) { + local = new HashMap<>(columnDefinitions.size()); + for (final ColumnDefinition columnDefinition : columnDefinitions) { + final String columnName = columnDefinition.getName(); + local.put(columnName, columnDefinition); + } + columnNameToDefinition = local; } - renameMap.put(filterColumnName, localColumnName); } } - - public List> columnDefinitions() { - return columnDefinitions; - } - - public Map renameMap() { - return renameMap; - } - - @Override - public void close() { - super.close(); - } + return local; } @Override public PushdownFilterContext makePushdownFilterContext( final WhereFilter filter, final List> filterSources) { - return new RegionedColumnSourcePushdownFilterContext(this, filter, filterSources); - } - - private abstract class JobBuilder { - - protected final WritableRowSet selection; - protected final int[] regionIndices; - protected final Consumer onError; - - private JobBuilder( - final WritableRowSet selection, - final int[] regionIndices, - final Consumer onError) { - this.selection = Objects.requireNonNull(selection); - this.regionIndices = Objects.requireNonNull(regionIndices); - this.onError = Objects.requireNonNull(onError); - } - - public final void iterateParallel( - final JobScheduler jobScheduler, - final JobScheduler.IterateResumeAction action) { - jobScheduler.iterateParallel( - ExecutionContext.getContext(), - this::log, - JobScheduler.DEFAULT_CONTEXT_FACTORY, - 0, - regionIndices.length, - action, - this::onJobsComplete, - this::jobsCleanup, - this::onJobsError); - } - - protected abstract LogOutput log(LogOutput output); - - protected abstract void onJobsComplete(); - - private void jobsCleanup() { - cleanupImpl(); - } - - private void onJobsError(Exception e) { - try (final SafeCloseable ignored = this::cleanupImpl) { - onError.accept(e); - } - } - - protected abstract void cleanupImpl(); - - abstract class JobRunner implements JobScheduler.IterateResumeAction { - protected final WhereFilter filter; - protected final boolean usePrev; - protected final io.deephaven.engine.table.impl.PushdownFilterContext context; - protected final JobScheduler jobScheduler; - - JobRunner( - final WhereFilter filter, - final boolean usePrev, - final io.deephaven.engine.table.impl.PushdownFilterContext context, - final JobScheduler jobScheduler) { - this.filter = Objects.requireNonNull(filter); - this.usePrev = usePrev; - this.context = Objects.requireNonNull(context); - this.jobScheduler = Objects.requireNonNull(jobScheduler); - } - - abstract class Job { - protected final int jobIndex; - private final Consumer nestedErrorConsumer; - private final Runnable locationResume; - protected final IncludedTableLocationEntry tle; - protected final WritableRowSet shiftedSubset; - private final AtomicBoolean closed; - - Job(final int jobIndex, final Consumer nestedErrorConsumer, final Runnable locationResume) { - this.jobIndex = jobIndex; - this.locationResume = Objects.requireNonNull(locationResume); - this.nestedErrorConsumer = Objects.requireNonNull(nestedErrorConsumer); - this.tle = orderedIncludedTableLocations.get(regionIndices[jobIndex]); - this.shiftedSubset = tle.subsetAndShiftIntoLocationSpace(selection); - this.closed = new AtomicBoolean(false); - } - - protected final void onCompleteSuccess() { - locationResume.run(); - close(); - } - - protected final void onError(Exception e) { - try (final SafeCloseable ignored = this::close) { - nestedErrorConsumer.accept(e); - } - } - - private void close() { - if (!closed.compareAndSet(false, true)) { - return; - } - shiftedSubset.close(); - } - } - } - } - - private final class EstimateJobBuilder extends JobBuilder { - private final LongConsumer onEstimateComplete; - private long minEstimate; - - EstimateJobBuilder(WritableRowSet selection, int[] regionIndices, LongConsumer onEstimateComplete, - Consumer onEstimateError) { - super(selection, regionIndices, onEstimateError); - this.onEstimateComplete = Objects.requireNonNull(onEstimateComplete); - this.minEstimate = Long.MAX_VALUE; - } - - public void iterateParallel( - final JobScheduler jobScheduler, - final WhereFilter filter, - final boolean usePrev, - final io.deephaven.engine.table.impl.PushdownFilterContext context) { - iterateParallel(jobScheduler, new EstimateJobRunner(filter, usePrev, context, jobScheduler)); - } + final List filterColumns = filter.getColumns(); + Require.eq(filterColumns.size(), "filterColumns.size()", + filterSources.size(), "filterSources.size()"); - private synchronized void addEstimate(final long estimate) { - if (estimate < minEstimate) { - minEstimate = estimate; - } - } - - @Override - protected LogOutput log(LogOutput output) { - return output.append("RegionedColumnSourceManager#estimatePushdownFilterCost"); - } - - @Override - protected void onJobsComplete() { - onEstimateComplete.accept(minEstimate); - } - - @Override - protected void cleanupImpl() { - selection.close(); - } - - final class EstimateJobRunner extends JobRunner { - EstimateJobRunner(WhereFilter filter, boolean usePrev, - io.deephaven.engine.table.impl.PushdownFilterContext context, - JobScheduler jobScheduler) { - super(filter, usePrev, context, jobScheduler); - } - - @Override - public void run(JobScheduler.JobThreadContext taskThreadContext, int index, - Consumer nestedErrorConsumer, Runnable resume) { - new EstimateJob(index, nestedErrorConsumer, resume).estimatePushdownFilterCost(); - } - - final class EstimateJob extends Job { - EstimateJob(int jobIndex, Consumer nestedErrorConsumer, Runnable locationResume) { - super(jobIndex, nestedErrorConsumer, locationResume); - } + final List> columnDefinitions = new ArrayList<>(filterSources.size()); + final Map renameMap = new HashMap<>(); - public void estimatePushdownFilterCost() { - tle.location.estimatePushdownFilterCost(filter, shiftedSubset, usePrev, context, jobScheduler, - this::onComplete, this::onError); - } + final IdentityHashMap, String> columnSourceToName = columnSourceToName(); + final Map> columnNameToDefinition = columnNameToDefinition(); - private void onComplete(long estimatedCost) { - addEstimate(estimatedCost); - onCompleteSuccess(); - } + for (int ii = 0; ii < filterColumns.size(); ii++) { + final String filterColumnName = filterColumns.get(ii); + final ColumnSource filterSource = filterSources.get(ii); + final String localColumnName = columnSourceToName.get(filterSource); + if (localColumnName == null) { + throw new IllegalArgumentException( + "No associated source for '" + filterColumnName + "' found in column sources"); } - } - } - - private final class PushdownJobBuilder extends JobBuilder { - private final Consumer onPushdownComplete; - - private final WritableRowSet[] matches; - private final WritableRowSet[] maybeMatches; - - public PushdownJobBuilder(WritableRowSet selection, int[] regionIndices, - Consumer onPushdownComplete, Consumer onPushdownError) { - super(selection, regionIndices, onPushdownError); - this.onPushdownComplete = Objects.requireNonNull(onPushdownComplete); - this.matches = new WritableRowSet[regionIndices.length]; - this.maybeMatches = new WritableRowSet[regionIndices.length]; - } - - public void iterateParallel( - final JobScheduler jobScheduler, - final WhereFilter filter, - final boolean usePrev, - final io.deephaven.engine.table.impl.PushdownFilterContext context, - final long costCeiling) { - iterateParallel(jobScheduler, new PushdownJobRunner(filter, usePrev, context, jobScheduler, costCeiling)); - } - - private void addResult(final int ix, final PushdownResult result) { - // Note: we are assuming that the lower-layer location pushdown logic is correct and using this assumption - // to build our results more efficiently because of that assumption. As such, we are destructuring the - // PushdownResult so that we can keep just the matches and maybeMatches and close selection since we don't - // _need_ it. - // - // If we ever need to be more defensive because we are seeing unexpected behavior, or we need to better - // validate this assumption for debugging purposes, it is easy to re-work this implementation so that this - // keeps the full PushdownResult and does a more thorough check in buildResults. - addResult(ix, result.match(), result.maybeMatch()); - - // Note: not closing result; we've already closed selection, and match / maybeMatch are now owned by this. - } + // Add the definition. + columnDefinitions.add(columnNameToDefinition.get(localColumnName)); - private synchronized void addResult( - final int jobIndex, - final WritableRowSet matchSubset, - final WritableRowSet maybeMatchSubset) { - // Note: we could consider a strategy where we incrementally compute the results via RowSet#insert or - // RowSetBuilderRandom#addRowSet. Without doing benchmarking, it is hard to say whether that would be a - // better approach. - // - // The justification for the current approach: - // 1. Very short time in addResult, meaning we won't block other jobs from completing / new jobs from - // running - // 2. In the case where the result represent the full selection, we can skip the building - this.matches[jobIndex] = matchSubset; - this.maybeMatches[jobIndex] = maybeMatchSubset; - } - - private synchronized PushdownResult buildResults() { - final long totalMatchSize = Stream.of(matches).mapToLong(RowSet::size).sum(); - final long totalMaybeMatchSize = Stream.of(maybeMatches).mapToLong(RowSet::size).sum(); - final long selectionSize = selection.size(); - if (totalMatchSize == selectionSize) { - Assert.eqZero(totalMaybeMatchSize, "totalMaybeMatchSize"); - return PushdownResult.allMatch(selection); - } - if (totalMaybeMatchSize == selectionSize) { - Assert.eqZero(totalMatchSize, "totalMatchSize"); - return PushdownResult.allMaybeMatch(selection); - } - if (totalMatchSize == 0 && totalMaybeMatchSize == 0) { - return PushdownResult.allNoMatch(selection); - } - // Note: it's not obvious what the best approach for building these RowSets is; that is, sequential - // insertion vs sequential builder. We know that the individual results are ordered and non-overlapping. - // If this becomes important, we can do more benchmarking. - try ( - final WritableRowSet match = RowSetFactory.unionInsert(Arrays.asList(matches)); - final WritableRowSet maybeMatch = RowSetFactory.unionInsert(Arrays.asList(maybeMatches))) { - return PushdownResult.of(selection, match, maybeMatch); - } - } - - @Override - protected LogOutput log(LogOutput output) { - return output.append("RegionedColumnSourceManager#pushdownFilter"); - } - - @Override - protected void onJobsComplete() { - onPushdownComplete.accept(buildResults()); - } - - @Override - protected void cleanupImpl() { - SafeCloseable.closeAll( - Stream.concat(Stream.of(selection), - Stream.concat( - Stream.of(matches), - Stream.of(maybeMatches)))); - } - - final class PushdownJobRunner extends JobRunner { - - private final long costCeiling; - - public PushdownJobRunner(WhereFilter filter, boolean usePrev, - io.deephaven.engine.table.impl.PushdownFilterContext context, - JobScheduler jobScheduler, long costCeiling) { - super(filter, usePrev, context, jobScheduler); - this.costCeiling = costCeiling; - } - - @Override - public void run(JobScheduler.JobThreadContext taskThreadContext, int index, - Consumer nestedErrorConsumer, Runnable resume) { - new PushdownJob(index, nestedErrorConsumer, resume).pushdownFilter(); - } - - final class PushdownJob extends Job { - public PushdownJob(int jobIndex, Consumer nestedErrorConsumer, Runnable locationResume) { - super(jobIndex, nestedErrorConsumer, locationResume); - } - - public void pushdownFilter() { - tle.location.pushdownFilter(filter, shiftedSubset, usePrev, context, costCeiling, jobScheduler, - this::onComplete, this::onError); - } - - private void onComplete(final PushdownResult pushdownResult) { - tle.unshiftIntoRegionSpace(pushdownResult); - addResult(jobIndex, pushdownResult); - onCompleteSuccess(); - } + // Add the rename (if needed) + if (localColumnName.equals(filterColumnName)) { + continue; } + renameMap.put(filterColumnName, localColumnName); } + return new RegionedPushdownFilterContextImpl(filter, filterSources, columnDefinitions, renameMap); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceObject.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceObject.java index 91faa98da79..975cab79d23 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceObject.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceObject.java @@ -77,7 +77,9 @@ public ColumnRegionObject makeRegion(@NotNull final ColumnDef + ": " + partitioningColumnValue + " is not a " + getType() + " at location " + locationKey); } // noinspection unchecked - return new ColumnRegionObject.Constant<>(PARAMETERS.regionMask, (DATA_TYPE) partitioningColumnValue); + return partitioningColumnValue == null + ? ColumnRegionObject.createNull(PARAMETERS.regionMask) + : new ColumnRegionObject.Constant<>(PARAMETERS.regionMask, (DATA_TYPE) partitioningColumnValue); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceShort.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceShort.java index 46ea7a56cc8..fc3ef1a4e38 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceShort.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedColumnSourceShort.java @@ -78,7 +78,9 @@ public ColumnRegionShort makeRegion(@NotNull final ColumnDefinition c "Unexpected partitioning column value type for " + columnDefinition.getName() + ": " + partitioningColumnValue + " is not a Short at location " + locationKey); } - return new ColumnRegionShort.Constant<>(regionMask(), unbox((Short) partitioningColumnValue)); + return partitioningColumnValue == null + ? ColumnRegionShort.createNull(regionMask()) + : new ColumnRegionShort.Constant<>(regionMask(), unbox((Short) partitioningColumnValue)); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownAction.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownAction.java new file mode 100644 index 00000000000..051b946b0c0 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownAction.java @@ -0,0 +1,164 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.util.SafeCloseable; + +import java.util.Objects; +import java.util.function.BooleanSupplier; +import java.util.function.Predicate; + +/** + * This class is used to represent a pushdown action that can be performed on a regioned column source. Note that the + * logic to perform this action is not contained in this class. + */ +public abstract class RegionedPushdownAction { + public static final EstimateContext DEFAULT_ESTIMATE_CONTEXT = () -> { + }; + public static final ActionContext DEFAULT_ACTION_CONTEXT = () -> { + }; + + /** + * A generic context for use in {@link RegionedPushdownFilterMatcher#estimatePushdownAction} + */ + public interface EstimateContext extends SafeCloseable { + } + + /** + * A generic context for use in {@link RegionedPushdownFilterMatcher#performPushdownAction} + */ + public interface ActionContext extends SafeCloseable { + } + + // Note: the disabled static configuration values are _not_ currently final and we must test them on each call + // to ensure that dynamic configuration changes are respected. + protected final BooleanSupplier disabled; + + private final long filterCost; + private final Predicate contextAllows; + private final Predicate locationAllows; + + private RegionedPushdownAction( + final BooleanSupplier disabled, + final long filterCost, + final Predicate contextAllows, + final Predicate locationAllows) { + this.disabled = Objects.requireNonNull(disabled); + this.filterCost = filterCost; + this.contextAllows = Objects.requireNonNull(contextAllows); + this.locationAllows = Objects.requireNonNull(locationAllows); + } + + /** + * The cost of performing this action. + */ + public long filterCost() { + return filterCost; + } + + /** + * Determine which actions are not specifically prohibited by configuration. + */ + public boolean allows( + final TableLocation location, + final RegionedPushdownFilterContext filterContext) { + return !disabled.getAsBoolean() + && contextAllows(filterContext) + && locationAllows(location); + } + + /** + * Determine which actions are not specifically prohibited by configuration and the cost ceiling + */ + public boolean allows( + final TableLocation location, + final RegionedPushdownFilterContext context, + final long costCeiling) { + return allows(location, context) && ceilingAllows(costCeiling); + } + + /** + * Determine if this action is allowed based on the cost ceiling. + */ + protected boolean ceilingAllows(final long costCeiling) { + return filterCost <= costCeiling; + } + + /** + * Determine if this action is allowed based on the current executed filter cost and the filter context predicate. + */ + protected boolean contextAllows(final RegionedPushdownFilterContext context) { + return context.executedFilterCost() < filterCost && contextAllows.test(context); + } + + /** + * Determine if this action is allowed based on the provided table location predicate. + */ + protected boolean locationAllows(final TableLocation location) { + return locationAllows.test(location); + } + + /** + * A pushdown action that is only valid when executed on a table location. + */ + public static class Location extends RegionedPushdownAction { + public Location( + final BooleanSupplier disabled, + final long filterCost, + final Predicate contextAllows, + final Predicate locationAllows) { + super(disabled, filterCost, contextAllows, locationAllows); + } + } + + /** + * A pushdown action that is only valid when executed on a column region. + */ + public static class Region extends RegionedPushdownAction { + private final Predicate> regionAllows; + + public Region( + final BooleanSupplier disabled, + final long filterCost, + final Predicate contextAllows, + final Predicate locationAllows, + final Predicate> regionAllows) { + super(disabled, filterCost, contextAllows, locationAllows); + this.regionAllows = Objects.requireNonNull(regionAllows); + } + + /** + * Determine if this action is allowed for the given table location, column region, and filter context. + */ + public boolean allows( + final TableLocation location, + final ColumnRegion region, + final RegionedPushdownFilterContext context) { + return !disabled.getAsBoolean() + && contextAllows(context) + && locationAllows(location) + && allows(region); + } + + /** + * Determine if this action is allowed for the given table location, column region, filter context, and cost + * ceiling. + */ + public boolean allows( + final TableLocation location, + final ColumnRegion region, + final RegionedPushdownFilterContext context, + final long costCeiling) { + return allows(location, region, context) && ceilingAllows(costCeiling); + } + + /** + * Determine if this action is allowed based on the column region predicate. + */ + private boolean allows(final ColumnRegion region) { + return regionAllows.test(region); + } + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterContext.java new file mode 100644 index 00000000000..f107af06e41 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterContext.java @@ -0,0 +1,25 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.impl.BasePushdownFilterContext; +import io.deephaven.engine.table.impl.locations.TableLocation; + +import java.util.List; +import java.util.Map; + +/** + * A pushdown filter context for regioned column sources that handles column name mappings and definitions. + */ +public interface RegionedPushdownFilterContext extends BasePushdownFilterContext { + List> columnDefinitions(); + + Map renameMap(); + + /** + * Create a wrapper of this context with the given table location set to the supplied value. + */ + RegionedPushdownFilterLocationContext withTableLocation(final TableLocation tableLocation); +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterContextImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterContextImpl.java new file mode 100644 index 00000000000..9751fb7497c --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterContextImpl.java @@ -0,0 +1,47 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.BasePushdownFilterContextImpl; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.select.WhereFilter; + +import java.util.List; +import java.util.Map; + +/** + * A pushdown filter context for regioned column sources that handles column name mappings and definitions. + */ +public class RegionedPushdownFilterContextImpl extends BasePushdownFilterContextImpl + implements RegionedPushdownFilterContext { + private final List> columnDefinitions; + private final Map renameMap; + + public RegionedPushdownFilterContextImpl( + final WhereFilter filter, + final List> columnSources, + final List> columnDefinitions, + final Map renameMap) { + super(filter, columnSources); + this.columnDefinitions = columnDefinitions; + this.renameMap = renameMap; + } + + public List> columnDefinitions() { + return columnDefinitions; + } + + public Map renameMap() { + return renameMap; + } + + /** + * Create a copy of this context with the given table location set to the supplied value. + */ + public RegionedPushdownFilterLocationContext withTableLocation(final TableLocation tableLocation) { + return new RegionedPushdownFilterLocationContext(this, tableLocation); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterLocationContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterLocationContext.java new file mode 100644 index 00000000000..0710beccdbf --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterLocationContext.java @@ -0,0 +1,107 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.impl.locations.TableLocation; +import io.deephaven.engine.table.impl.select.WhereFilter; + +import java.util.List; +import java.util.Map; + +/** + * A wrapper class for RegionedPushdownFilterContext that additionally provides access to a table location. + */ +public class RegionedPushdownFilterLocationContext implements RegionedPushdownFilterContext { + private final RegionedPushdownFilterContext wrapped; + private final TableLocation tableLocation; + + public RegionedPushdownFilterLocationContext( + final RegionedPushdownFilterContext wrapped, + final TableLocation tableLocation) { + this.wrapped = wrapped; + this.tableLocation = tableLocation; + } + + @Override + public void close() { + // Not releasing or modifying the wrapped context. + } + + public TableLocation tableLocation() { + return tableLocation; + } + + @Override + public WhereFilter filter() { + return wrapped.filter(); + } + + @Override + public List> columnSources() { + return wrapped.columnSources(); + } + + @Override + public boolean supportsChunkFiltering() { + return wrapped.supportsChunkFiltering(); + } + + @Override + public boolean supportsMetadataFiltering() { + return wrapped.supportsMetadataFiltering(); + } + + @Override + public boolean supportsInMemoryDataIndexFiltering() { + return wrapped.supportsInMemoryDataIndexFiltering(); + } + + @Override + public boolean supportsDeferredDataIndexFiltering() { + return wrapped.supportsDeferredDataIndexFiltering(); + } + + @Override + public WhereFilter filterForMetadataFiltering() { + return wrapped.filterForMetadataFiltering(); + } + + @Override + public FilterNullBehavior filterNullBehavior() { + return wrapped.filterNullBehavior(); + } + + @Override + public UnifiedChunkFilter createChunkFilter(int maxChunkSize) { + return wrapped.createChunkFilter(maxChunkSize); + } + + @Override + public long executedFilterCost() { + return wrapped.executedFilterCost(); + } + + @Override + public void updateExecutedFilterCost(long executedFilterCost) { + // These wrapped context are transient, should not be called. + throw new UnsupportedOperationException("Should not update executed filter cost on wrapped context"); + } + + @Override + public List> columnDefinitions() { + return wrapped.columnDefinitions(); + } + + @Override + public Map renameMap() { + return wrapped.renameMap(); + } + + @Override + public RegionedPushdownFilterLocationContext withTableLocation(TableLocation tableLocation) { + return new RegionedPushdownFilterLocationContext(wrapped, tableLocation); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterMatcher.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterMatcher.java new file mode 100644 index 00000000000..6e3df33bb9b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownFilterMatcher.java @@ -0,0 +1,72 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.table.impl.PushdownFilterContext; +import io.deephaven.engine.table.impl.PushdownFilterMatcher; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.select.WhereFilter; + +import java.util.List; + +public interface RegionedPushdownFilterMatcher extends PushdownFilterMatcher { + /** + * Get the list of pushdown actions supported by this matcher. Defaults to an empty list. + */ + default List supportedActions() { + // Default to an empty list. + return List.of(); + } + + /** + * Create a context for estimating the cost for the filter and filter context. This context can be used to provide + * additional information to the {@link #estimatePushdownAction} function + */ + default RegionedPushdownAction.EstimateContext makeEstimateContext( + final WhereFilter filter, + final PushdownFilterContext context) { + return RegionedPushdownAction.DEFAULT_ESTIMATE_CONTEXT; + } + + /** + * Given a list of actions, estimate the cost of the next pushdown action. This is not always the first action in + * the list because this matcher may not support every allowed action. + */ + default long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + return Long.MAX_VALUE; + } + + /** + * Create a context for processing the given filter and filter context. This context can be used to provide + * additional information to the {@link #performPushdownAction} function + */ + + default RegionedPushdownAction.ActionContext makeActionContext( + final WhereFilter filter, + final PushdownFilterContext context) { + return RegionedPushdownAction.DEFAULT_ACTION_CONTEXT; + } + + /** + * Perform the pushdown action for the given filter and filter context. + */ + default PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + // Return a copy of the input because the caller will close the input. + return input.copy(); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownHelper.java new file mode 100644 index 00000000000..786e4e0d364 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/sources/regioned/RegionedPushdownHelper.java @@ -0,0 +1,65 @@ +// +// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.sources.regioned; + +import io.deephaven.base.verify.Assert; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.table.impl.PushdownResult; +import io.deephaven.engine.table.impl.util.JobScheduler; + +import java.util.*; +import java.util.stream.Stream; + +public class RegionedPushdownHelper { + /** + * The context for regioned pushdown parallel JobScheduler operations. + */ + public static class RegionThreadContext implements JobScheduler.JobThreadContext { + RowSet shiftedRowSet; + + @Override + public void close() { + try (RowSet ignored = shiftedRowSet) { + } + } + + public void reset() { + if (shiftedRowSet != null) { + shiftedRowSet.close(); + shiftedRowSet = null; + } + } + } + + /** + * Combine the results from multiple regioned pushdown operations into a unified result. + */ + public static PushdownResult buildResults( + final WritableRowSet[] matches, + final WritableRowSet[] maybeMatches, + final RowSet selection) { + final long totalMatchSize = Stream.of(matches).mapToLong(RowSet::size).sum(); + final long totalMaybeMatchSize = Stream.of(maybeMatches).mapToLong(RowSet::size).sum(); + final long selectionSize = selection.size(); + if (totalMatchSize == selectionSize) { + Assert.eqZero(totalMaybeMatchSize, "totalMaybeMatchSize"); + return PushdownResult.allMatch(selection); + } + if (totalMaybeMatchSize == selectionSize) { + Assert.eqZero(totalMatchSize, "totalMatchSize"); + return PushdownResult.allMaybeMatch(selection); + } + if (totalMatchSize == 0 && totalMaybeMatchSize == 0) { + return PushdownResult.allNoMatch(selection); + } + // Note: it's not obvious what the best approach for building these RowSets is; that is, sequential + // insertion vs sequential builder. We know that the individual results are ordered and non-overlapping. + // If this becomes important, we can do more benchmarking. + try ( + final WritableRowSet match = RowSetFactory.unionInsert(Arrays.asList(matches)); + final WritableRowSet maybeMatch = RowSetFactory.unionInsert(Arrays.asList(maybeMatches))) { + return PushdownResult.of(selection, match, maybeMatch); + } + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java index 3e70273de9c..657dc9d7f6c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java @@ -2665,6 +2665,10 @@ private void testRowKeyAgnosticColumnSource( preFilter.reset(); postFilter.reset(); + // The rowset capturing filters ignore some optimization opportunities (like chunk filtering) so run these + // again without wrappers and compare the results. + assertTableEquals(res0, source.where(filterAllPass)); + final RowSetCapturingFilter filter1 = new ParallelizedRowSetCapturingFilter(RawString.of(filterNonePass)); // force pre and post filters to run when expected using barriers @@ -2677,6 +2681,8 @@ private void testRowKeyAgnosticColumnSource( assertEquals(0, postFilter.numRowsProcessed()); // No rows passed assertEquals(0, res1.size()); + + assertTableEquals(res1, source.where(filterNonePass)); } @Test diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTableNoMocks.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTableNoMocks.java index a1a29958ac2..fe0677c4e00 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTableNoMocks.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestPartitionAwareSourceTableNoMocks.java @@ -421,8 +421,8 @@ public void testDeferredPartitioningFilterSplitsBarrier() { Assert.eq(filter0.numRowsProcessed(), "filter0.numRowsProcessed()", 2 * partitionSize); // ensure we see the barrier partition filter as filtering only the partitioned rows Assert.eq(filter1.numRowsProcessed(), "filter1.numRowsProcessed()", 4); - // however, the respects barrier could not be lifted, so it should match result of filter0 - Assert.eq(filter2.numRowsProcessed(), "filter2.numRowsProcessed()", 2 * (partitionSize / 2)); + // the respects barrier could not be lifted but operates on constant column regions (vs. rows) + Assert.eq(filter2.numRowsProcessed(), "filter2.numRowsProcessed()", 2); Assert.eq(res0.size(), "res0.size()", partitionSize / 2); } diff --git a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java index 642829f1e60..0ddfb25293d 100644 --- a/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java +++ b/extensions/parquet/table/src/main/java/io/deephaven/parquet/table/location/ParquetTableLocation.java @@ -16,7 +16,6 @@ import io.deephaven.engine.primitive.iterator.CloseableIterator; import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.table.impl.filter.ExtractFilterWithoutBarriers; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.BasePushdownFilterContext; import io.deephaven.engine.table.impl.PushdownFilterContext; @@ -26,13 +25,11 @@ import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; import io.deephaven.engine.table.impl.chunkfilter.LongChunkMatchFilterFactory; import io.deephaven.engine.table.impl.dataindex.StandaloneDataIndex; +import io.deephaven.engine.table.impl.filter.ExtractFilterWithoutBarriers; import io.deephaven.engine.table.impl.locations.*; import io.deephaven.engine.table.impl.locations.impl.AbstractTableLocation; +import io.deephaven.engine.table.impl.sources.regioned.*; import io.deephaven.engine.table.impl.select.*; -import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSource; -import io.deephaven.engine.table.impl.sources.regioned.RegionedColumnSourceManager; -import io.deephaven.engine.table.impl.sources.regioned.RegionedPageStore; -import io.deephaven.engine.table.impl.util.JobScheduler; import io.deephaven.engine.table.vectors.ColumnVectors; import io.deephaven.internal.log.LoggerFactory; import io.deephaven.io.logger.Logger; @@ -67,10 +64,6 @@ import java.nio.file.Path; import java.time.Instant; import java.util.*; -import java.util.function.BooleanSupplier; -import java.util.function.Consumer; -import java.util.function.LongConsumer; -import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -461,6 +454,45 @@ private static Table readDataIndexTable( // region Pushdown Filtering + private static final RegionedPushdownAction.Location ParquetRowGroupMetadata = + new RegionedPushdownAction.Location( + () -> QueryTable.DISABLE_WHERE_PUSHDOWN_PARQUET_ROW_GROUP_METADATA, + PushdownResult.METADATA_STATS_COST, + BasePushdownFilterContext::supportsMetadataFiltering, + (tl) -> ((ParquetTableLocation) tl).supportsMetadataFiltering()); + + private static final RegionedPushdownAction.Location InMemoryDataIndex = + new RegionedPushdownAction.Location( + () -> QueryTable.DISABLE_WHERE_PUSHDOWN_DATA_INDEX, + PushdownResult.IN_MEMORY_DATA_INDEX_COST, + BasePushdownFilterContext::supportsInMemoryDataIndexFiltering, + (tl) -> ((ParquetTableLocation) tl).supportsInMemoryDataIndexFiltering()); + + private static final RegionedPushdownAction.Location ParquetDictionary = + new RegionedPushdownAction.Location( + () -> QueryTable.DISABLE_WHERE_PUSHDOWN_PARQUET_DICTIONARY, + PushdownResult.DICTIONARY_DATA_COST, + BasePushdownFilterContext::supportsChunkFiltering, + (tl) -> ((ParquetTableLocation) tl).supportsDictionaryFiltering()); + + private static final RegionedPushdownAction.Location DeferredDataIndex = + new RegionedPushdownAction.Location( + () -> QueryTable.DISABLE_WHERE_PUSHDOWN_DATA_INDEX, + PushdownResult.DEFERRED_DATA_INDEX_COST, + BasePushdownFilterContext::supportsDeferredDataIndexFiltering, + (tl) -> ((ParquetTableLocation) tl).supportsDeferredDataIndexFiltering()); + + private static final List supportedActions = List.of( + ParquetRowGroupMetadata, + InMemoryDataIndex, + ParquetDictionary, + DeferredDataIndex); + + @Override + public List supportedActions() { + return supportedActions; + } + /** * Checks if the column has a dictionary page. * @@ -483,77 +515,81 @@ private boolean hasDictionaryPage(final String parquetColumnName, final ColumnDe && dictionaryChunk.size() > 0; } - @Override - public void estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final JobScheduler jobScheduler, - final LongConsumer onComplete, - final Consumer onError) { - onComplete.accept(estimatePushdownFilterCost(filter, selection, usePrev, context)); - } - - private long estimatePushdownFilterCost( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context) { - if (selection.isEmpty()) { - // If the selection is empty, we can skip all pushdown filtering. - log.warn().append("Estimate pushdown filter cost called with empty selection for table ") - .append(getTableKey()).endl(); - return Long.MAX_VALUE; + public static class EstimateContext implements RegionedPushdownAction.EstimateContext { + enum ResolveState { + RESOLVED, FAILED } - final RegionedColumnSourceManager.RegionedColumnSourcePushdownFilterContext ctx = - (RegionedColumnSourceManager.RegionedColumnSourcePushdownFilterContext) context; - - final List allowedModes = PushdownMode.costSortedValues() - .stream() - .filter(mode -> mode.allows(this, ctx)) - .collect(Collectors.toList()); + private final ResolveState resolveState; + private final String[] parquetColumnNames; - if (allowedModes.isEmpty()) { - return Long.MAX_VALUE; + private EstimateContext( + final ResolveState resolveState, + final String[] parquetColumnNames) { + this.resolveState = resolveState; + this.parquetColumnNames = parquetColumnNames; } + @Override + public void close() {} + } + + @Override + public RegionedPushdownAction.EstimateContext makeEstimateContext( + final WhereFilter filter, + final PushdownFilterContext filterContext) { + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) filterContext; + + // We must have an initialized location to create this estimate context. initialize(); - final Optional> maybeResolvedColumns = resolveColumns(filter, ctx.renameMap()); + final Optional> maybeResolvedColumns = resolveColumns(filter, filterCtx.renameMap()); if (maybeResolvedColumns.isEmpty()) { - // One or more columns could not be resolved, so no benefit to pushing down. - return Long.MAX_VALUE; + return new EstimateContext(EstimateContext.ResolveState.FAILED, null); } + final List resolvedColumnsInfo = maybeResolvedColumns.get(); // We have verified these columns are not nested. final String[] parquetColumnNames = resolvedColumnsInfo.stream() .map(resolvedColumn -> resolvedColumn.columnPath.get(0)) .toArray(String[]::new); + return new EstimateContext(EstimateContext.ResolveState.RESOLVED, parquetColumnNames); + } + + @Override + public long estimatePushdownAction( + final List actions, + final WhereFilter filter, + final RowSet selection, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.EstimateContext estimateContext) { + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) filterContext; + final EstimateContext estimateCtx = (EstimateContext) estimateContext; + + if (estimateCtx.resolveState == EstimateContext.ResolveState.FAILED) { + // One or more columns could not be resolved, so no benefit to pushing down. + return Long.MAX_VALUE; + } // Apply a more specific check that depends on materializing parquet metadata - for (final PushdownMode mode : allowedModes) { + for (final RegionedPushdownAction action : actions) { final boolean isApplicable; - switch (mode) { - case ParquetRowGroupMetadata: - // Note: it should be possible to check if there is any statistics - isApplicable = true; - break; - case InMemoryDataIndex: - isApplicable = hasCachedDataIndex(parquetColumnNames); - break; - case ParquetDictionary: - isApplicable = hasDictionaryPage(parquetColumnNames[0], ctx.columnDefinitions().get(0)); - break; - case DeferredDataIndex: - isApplicable = hasDataIndex(parquetColumnNames); - break; - default: - throw new IllegalStateException("Unexpected value: " + mode); + if (action == ParquetRowGroupMetadata) { + // Note: it should be possible to check if there is any statistics + isApplicable = true; + } else if (action == InMemoryDataIndex) { + isApplicable = hasCachedDataIndex(estimateCtx.parquetColumnNames); + } else if (action == ParquetDictionary) { + isApplicable = + hasDictionaryPage(estimateCtx.parquetColumnNames[0], filterCtx.columnDefinitions().get(0)); + } else if (action == DeferredDataIndex) { + isApplicable = hasDataIndex(estimateCtx.parquetColumnNames); + } else { + throw new IllegalStateException("Unexpected value: " + action); } if (isApplicable) { - return mode.filterCost(); + return action.filterCost(); } } @@ -561,6 +597,105 @@ private long estimatePushdownFilterCost( return Long.MAX_VALUE; // No benefit to pushing down. } + public static class ActionContext implements RegionedPushdownAction.ActionContext { + enum ResolveState { + RESOLVED, FAILED + } + + private final ResolveState resolveState; + private final String[] parquetColumnNames; + private final List columnIndices; + + private ActionContext( + final ResolveState resolveState, + final String[] parquetColumnNames, + final List columnIndices) { + this.resolveState = resolveState; + this.parquetColumnNames = parquetColumnNames; + this.columnIndices = columnIndices; + } + + @Override + public void close() {} + } + + @Override + public RegionedPushdownAction.ActionContext makeActionContext( + final WhereFilter filter, + final PushdownFilterContext filterContext) { + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) filterContext; + + // We must have an initialized location to create this action context. + initialize(); + + final Map renameMap = filterCtx.renameMap(); + final Optional> maybeResolvedColumns = resolveColumns(filter, renameMap); + if (maybeResolvedColumns.isEmpty()) { + return new ActionContext(ActionContext.ResolveState.FAILED, null, null); + } + + final List resolvedColumnsInfo = maybeResolvedColumns.get(); + + // We have verified these columns are not nested. + final int numColumns = resolvedColumnsInfo.size(); + final String[] parquetColumnNames = new String[numColumns]; + final List columnIndices = new ArrayList<>(numColumns); + + for (int i = 0; i < numColumns; i++) { + final ResolvedColumnInfo resolvedColumn = resolvedColumnsInfo.get(i); + parquetColumnNames[i] = resolvedColumn.columnPath.get(0); + columnIndices.add(resolvedColumn.columnIndex); + } + return new ActionContext(ActionContext.ResolveState.RESOLVED, parquetColumnNames, columnIndices); + } + + @Override + public PushdownResult performPushdownAction( + final RegionedPushdownAction action, + final WhereFilter filter, + final RowSet selection, + final PushdownResult input, + final boolean usePrev, + final PushdownFilterContext filterContext, + final RegionedPushdownAction.ActionContext actionContext) { + final RegionedPushdownFilterContext filterCtx = (RegionedPushdownFilterContext) filterContext; + final ActionContext actionCtx = (ActionContext) actionContext; + + if (actionCtx.resolveState == ActionContext.ResolveState.FAILED) { + // One or more columns could not be resolved, so return the input + return input.copy(); + } + + if (action == ParquetRowGroupMetadata) { + return pushdownRowGroupMetadata(selection, filterCtx.filterForMetadataFiltering(), actionCtx.columnIndices, + input); + } + if (action == InMemoryDataIndex) { + final BasicDataIndex dataIndex = + hasCachedDataIndex(actionCtx.parquetColumnNames) ? getDataIndex(actionCtx.parquetColumnNames) + : null; + if (dataIndex == null) { + return input.copy(); + } + return pushdownDataIndex(selection, filter, filterCtx.renameMap(), dataIndex, input); + } + if (action == ParquetDictionary) { + if (!hasDictionaryPage(actionCtx.parquetColumnNames[0], filterCtx.columnDefinitions().get(0))) { + return input.copy(); + } + return pushdownFilterDictionary(selection, filterCtx, actionCtx.parquetColumnNames, input); + } + if (action == DeferredDataIndex) { + final BasicDataIndex dataIndex = + hasDataIndex(actionCtx.parquetColumnNames) ? getDataIndex(actionCtx.parquetColumnNames) : null; + if (dataIndex == null) { + return input.copy(); + } + return pushdownDataIndex(selection, filter, filterCtx.renameMap(), dataIndex, input); + } + throw new IllegalStateException("Unexpected value: " + action); + } + /** * A helper class to hold the resolved column paths and their corresponding column indices. */ @@ -667,111 +802,6 @@ private Optional> resolveColumns( return Optional.of(resolvedColumns); } - @Override - public void pushdownFilter( - final WhereFilter filter, - final RowSet selection, - final boolean usePrev, - final PushdownFilterContext context, - final long costCeiling, - final JobScheduler jobScheduler, - final Consumer onComplete, - final Consumer onError) { - if (selection.isEmpty()) { - log.warn().append("Pushdown filter called with empty selection for table ").append(getTableKey()).endl(); - onComplete.accept(PushdownResult.allNoMatch(selection)); - return; - } - - final RegionedColumnSourceManager.RegionedColumnSourcePushdownFilterContext ctx = - (RegionedColumnSourceManager.RegionedColumnSourcePushdownFilterContext) context; - - // Initialize the pushdown result with the selection rowset as "maybe" rows - PushdownResult result = PushdownResult.allMaybeMatch(selection); - - final List allowedModes = PushdownMode.costSortedValues() - .stream() - .filter(mode -> mode.allows(this, ctx, costCeiling)) - .collect(Collectors.toList()); - - if (allowedModes.isEmpty()) { - onComplete.accept(result); - return; - } - - initialize(); - - final Map renameMap = ctx.renameMap(); - final Optional> maybeResolvedColumns = resolveColumns(filter, renameMap); - if (maybeResolvedColumns.isEmpty()) { - // One or more columns could not be resolved, so we return all rows as "maybe" rows. - onComplete.accept(result); - return; - } - final List resolvedColumnsInfo = maybeResolvedColumns.get(); - - // We have verified these columns are not nested. - final int numColumns = resolvedColumnsInfo.size(); - final String[] parquetColumnNames = new String[numColumns]; - final List columnIndices = new ArrayList<>(numColumns); - - for (int i = 0; i < numColumns; i++) { - final ResolvedColumnInfo resolvedColumn = resolvedColumnsInfo.get(i); - parquetColumnNames[i] = resolvedColumn.columnPath.get(0); - columnIndices.add(resolvedColumn.columnIndex); - } - - for (final PushdownMode mode : allowedModes) { - switch (mode) { - case ParquetRowGroupMetadata: { - try (final PushdownResult ignored = result) { - result = pushdownRowGroupMetadata(selection, ctx.filterForMetadataFiltering(), columnIndices, - result); - } - break; - } - case InMemoryDataIndex: { - final BasicDataIndex dataIndex = - hasCachedDataIndex(parquetColumnNames) ? getDataIndex(parquetColumnNames) : null; - if (dataIndex == null) { - continue; - } - try (final PushdownResult ignored = result) { - result = pushdownDataIndex(selection, filter, renameMap, dataIndex, result); - } - break; - } - case ParquetDictionary: { - if (!hasDictionaryPage(parquetColumnNames[0], ctx.columnDefinitions().get(0))) { - continue; - } - try (final PushdownResult ignored = result) { - result = pushdownFilterDictionary(selection, ctx, parquetColumnNames, result); - } - break; - } - case DeferredDataIndex: { - final BasicDataIndex dataIndex = - hasDataIndex(parquetColumnNames) ? getDataIndex(parquetColumnNames) : null; - if (dataIndex == null) { - continue; - } - try (final PushdownResult ignored = result) { - result = pushdownDataIndex(selection, filter, renameMap, dataIndex, result); - } - break; - } - default: - throw new IllegalStateException("Unexpected value: " + mode); - } - if (result.maybeMatch().isEmpty()) { - // No maybe rows remaining, so no reason to continue filtering. - break; - } - } - onComplete.accept(result); - } - // --------------------------------------------------------------------------------------------------------------- // The following should be _cheap_ checks that don't require materializing Parquet metadata to check. // --------------------------------------------------------------------------------------------------------------- @@ -783,6 +813,7 @@ private boolean supportsMetadataFiltering() { } private boolean supportsDictionaryFiltering() { + return true; } @@ -796,86 +827,6 @@ private boolean supportsDeferredDataIndexFiltering() { // --------------------------------------------------------------------------------------------------------------- - enum PushdownMode { - // @formatter:off - ParquetRowGroupMetadata( - () -> QueryTable.DISABLE_WHERE_PUSHDOWN_PARQUET_ROW_GROUP_METADATA, - PushdownResult.METADATA_STATS_COST, - BasePushdownFilterContext::supportsMetadataFiltering, - ParquetTableLocation::supportsMetadataFiltering), - InMemoryDataIndex( - () -> QueryTable.DISABLE_WHERE_PUSHDOWN_DATA_INDEX, - PushdownResult.IN_MEMORY_DATA_INDEX_COST, - BasePushdownFilterContext::supportsInMemoryDataIndexFiltering, - ParquetTableLocation::supportsInMemoryDataIndexFiltering), - ParquetDictionary( - () -> QueryTable.DISABLE_WHERE_PUSHDOWN_PARQUET_DICTIONARY, - PushdownResult.DICTIONARY_DATA_COST, - BasePushdownFilterContext::supportsDictionaryFiltering, - ParquetTableLocation::supportsDictionaryFiltering), - DeferredDataIndex( - () -> QueryTable.DISABLE_WHERE_PUSHDOWN_DATA_INDEX, - PushdownResult.DEFERRED_DATA_INDEX_COST, - BasePushdownFilterContext::supportsDeferredDataIndexFiltering, - ParquetTableLocation::supportsDeferredDataIndexFiltering); - // @formatter:on - - public static List costSortedValues() { - // Since the costs are not currently dynamic, we'll just use the fact that the enum values are already in - // cost order (validated by unit test). - return Arrays.asList(PushdownMode.values()); - } - - // Note: the disabled static configuration values are _not_ currently final. While ill-advised to change them in - // this way, it is better that we retain the current behavior until a better pattern can be established. - // private final boolean disabled; - private final BooleanSupplier disabled; - private final long filterCost; - private final Predicate contextAllows; - private final Predicate locationAllows; - - PushdownMode( - final BooleanSupplier disabled, - final long filterCost, - final Predicate contextAllows, - final Predicate locationAllows) { - this.disabled = Objects.requireNonNull(disabled); - this.filterCost = filterCost; - this.contextAllows = Objects.requireNonNull(contextAllows); - this.locationAllows = Objects.requireNonNull(locationAllows); - } - - public long filterCost() { - return filterCost; - } - - public boolean allows( - final ParquetTableLocation location, - final BasePushdownFilterContext context) { - return !disabled.getAsBoolean() - && contextAllows(context) - && locationAllows(location); - } - - public boolean allows( - final ParquetTableLocation location, - final BasePushdownFilterContext context, - final long costCeiling) { - return allows(location, context) && ceilingAllows(costCeiling); - } - - private boolean ceilingAllows(final long costCeiling) { - return filterCost <= costCeiling; - } - - private boolean contextAllows(final BasePushdownFilterContext context) { - return context.executedFilterCost() < filterCost && contextAllows.test(context); - } - - private boolean locationAllows(final ParquetTableLocation location) { - return locationAllows.test(location); - } - } /** * Consumer for row groups and row sets. */ @@ -998,7 +949,7 @@ private PushdownResult pushdownRowGroupMetadata( @NotNull private PushdownResult pushdownFilterDictionary( final RowSet selection, - final RegionedColumnSourceManager.RegionedColumnSourcePushdownFilterContext ctx, + final RegionedPushdownFilterContext ctx, final String[] parquetColumnNames, final PushdownResult result) { @@ -1128,12 +1079,11 @@ private PushdownResult pushdownFilterDictionary( } } - /** * Apply the filter to the data index table and return the result. */ @NotNull - private PushdownResult pushdownDataIndex( + public static PushdownResult pushdownDataIndex( final RowSet selection, final WhereFilter filter, final Map renameMap, diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableFilterTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableFilterTest.java index e83d2406d38..047eacf0ecf 100644 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableFilterTest.java +++ b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/ParquetTableFilterTest.java @@ -55,8 +55,7 @@ import static io.deephaven.engine.util.TableTools.intCol; import static io.deephaven.engine.util.TableTools.newTable; import static io.deephaven.engine.util.TableTools.stringCol; -import static io.deephaven.parquet.table.ParquetTools.readTable; -import static io.deephaven.parquet.table.ParquetTools.writeTable; +import static io.deephaven.parquet.table.ParquetTools.*; import static io.deephaven.time.DateTimeUtils.parseInstant; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -69,6 +68,7 @@ public final class ParquetTableFilterTest { private static final ParquetInstructions EMPTY = ParquetInstructions.EMPTY; private static File rootFile; + private boolean whereDataIndexEnabled; @Rule public final EngineCleanup framework = new EngineCleanup(); @@ -81,11 +81,13 @@ public void setUp() { } // noinspection ResultOfMethodCallIgnored rootFile.mkdirs(); + whereDataIndexEnabled = QueryTable.USE_DATA_INDEX_FOR_WHERE; } @After public void tearDown() { FileUtils.deleteRecursively(rootFile); + QueryTable.USE_DATA_INDEX_FOR_WHERE = whereDataIndexEnabled; } private Table[] splitTable(final Table source, int numSplits, boolean randomSizes) { @@ -950,13 +952,13 @@ public void flatPartitionsLoadedDataIndexTest() { diskRowsProcessedMergedIndex.add(filter.numRowsProcessed()); } - // Verify that the merged indexes processed strictly fewer rows the disk table with location indexes. + // Verify that the merged indexes processed strictly fewer rows than the mem table. for (int i = 0; i < filters.size(); i++) { Assert.assertTrue( "Merged indexes did not process fewer rows than Location indexes: " + i + ", Merged index rows processed: " + diskRowsProcessedMergedIndex.get(i) - + ", Location index rows processed: " + diskRowsProcessedLocationIndexes.get(i), - diskRowsProcessedMergedIndex.get(i) < diskRowsProcessedLocationIndexes.get(i)); + + ", Mem rows processed: " + memRowsProcessed.get(i), + diskRowsProcessedMergedIndex.get(i) < memRowsProcessed.get(i)); } } @@ -2094,4 +2096,135 @@ public void testLocationDataIndexWithFilterBarriers() { assertEquals(26430, result.size()); allFilters.forEach(RowSetCapturingFilter::reset); } + + @Test + public void testPartitioningTableColumnRegions() { + // Partitioning columns are automatically added to a data index. We have to disable use of the data index + // in order to test constant and null column region pushdown features. + QueryTable.USE_DATA_INDEX_FOR_WHERE = false; + + QueryScope.addParam("symList", List.of("alpha", "bravo", "charlie", "delta", "echo", "foxtrot")); + final Table tmpTable = TableTools.emptyTable(100_000).update( + "Sym = i % 7 == 6 ? (String)null : (String)symList.get(i % 7)", + "A = i % 97 == 0 ? null : i % 97", + "B = i % 11 == 0 ? null : i % 11", + "C = i"); + final PartitionedTable partitionedTable = tmpTable.partitionBy("Sym", "A"); + + final String destPath = Path.of(rootFile.getPath(), "partitioningTableColumnRegions").toString(); + final ParquetInstructions writeInstructions = new ParquetInstructions.Builder() + .build(); + writeKeyValuePartitionedTable(partitionedTable, destPath, writeInstructions); + + // Coalesce the table to prevent PAST optimizations. + final Table diskTable = ParquetTools.readTable(destPath).coalesce(); + final Table memTable = diskTable.select(); + + final Filter filterSym = RawString.of("Sym in `alpha`, `bravo`"); + final Filter filterSymConditional = RawString.of("true && (Sym == `alpha` || Sym == `bravo`)"); + final Filter filterA = RawString.of("A < 50"); + final Filter filterAConditional = RawString.of("true && A < 50"); + final Filter filterB = RawString.of("B < 5"); + + + // Create some capturing filters to verify the row sets being passed through the filter chain. + try (final RowSetCapturingFilter capturingFilterSym = new ParallelizedRowSetCapturingFilter(filterSym); + final RowSetCapturingFilter capturingFilterSymConditional = + new ParallelizedRowSetCapturingFilter(filterSymConditional); + final RowSetCapturingFilter capturingFilterA = new ParallelizedRowSetCapturingFilter(filterA); + final RowSetCapturingFilter capturingFilterAConditional = + new ParallelizedRowSetCapturingFilter(filterAConditional); + final RowSetCapturingFilter capturingFilterB = new ParallelizedRowSetCapturingFilter(filterB)) { + + final List allFilters = List.of(capturingFilterSym, capturingFilterSymConditional, + capturingFilterA, capturingFilterAConditional, capturingFilterB); + + Table result; + + result = diskTable.where(capturingFilterSym); + + // 583 is explained as follows. There are 6 * 97 = 582 regions where Sym is not-null which are tested. + // There are 97 null regions for Sym which are all covered by a single test (to determine filter null + // behavior). + assertEquals(583, capturingFilterSym.numRowsProcessed()); + assertEquals(28572, result.size()); + + // Use the unwrapped filter to test other optimization paths (i.e. chunk filtering) and assert equality. + assertTableEquals(result, memTable.where(filterSym)); + assertTableEquals(result, diskTable.where(filterSym)); + + allFilters.forEach(RowSetCapturingFilter::reset); + + ////////////////////////////////////////////////////// + + result = diskTable.where(capturingFilterSymConditional); + + assertEquals(583, capturingFilterSymConditional.numRowsProcessed()); + assertEquals(28572, result.size()); + + // Use the unwrapped filter to test other optimization paths (i.e. chunk filtering) and assert equality. + assertTableEquals(result, memTable.where(filterSymConditional)); + assertTableEquals(result, diskTable.where(filterSymConditional)); + + allFilters.forEach(RowSetCapturingFilter::reset); + + ////////////////////////////////////////////////////// + + result = diskTable.where(capturingFilterA).coalesce(); + + // 673 is explained as follows. There are 7 * 96 = 672 regions where A is not null. There are 6 null + // regions for A, which are covered by a single test for filter null behavior. + assertEquals(673, capturingFilterA.numRowsProcessed()); + assertEquals(51550, result.size()); + + // Use the unwrapped filter to test other optimization paths (i.e. chunk filtering) and assert equality. + assertTableEquals(result, memTable.where(filterA)); + assertTableEquals(result, diskTable.where(filterA)); + + allFilters.forEach(RowSetCapturingFilter::reset); + + ////////////////////////////////////////////////////// + + result = diskTable.where(capturingFilterAConditional).coalesce(); + + assertEquals(673, capturingFilterAConditional.numRowsProcessed()); + assertEquals(51550, result.size()); + + // Use the unwrapped filter to test other optimization paths (i.e. chunk filtering) and assert equality. + assertTableEquals(result, memTable.where(filterAConditional)); + assertTableEquals(result, diskTable.where(filterAConditional)); + + allFilters.forEach(RowSetCapturingFilter::reset); + + ////////////////////////////////////////////////////// + + result = diskTable.where(capturingFilterB); + + // All rows to be tested. + assertEquals(100000, capturingFilterB.numRowsProcessed()); + assertEquals(45455, result.size()); + + // Use the unwrapped filter to test other optimization paths (i.e. chunk filtering) and assert equality. + assertTableEquals(result, memTable.where(filterB)); + assertTableEquals(result, diskTable.where(filterB)); + + allFilters.forEach(RowSetCapturingFilter::reset); + + ////////////////////////////////////////////////////// + + result = diskTable.where(Filter.and(capturingFilterSym, capturingFilterA)).coalesce(); + + // All regions tested for Sym match + assertEquals(583, capturingFilterSym.numRowsProcessed()); + // A subset of regions tested for A match + assertEquals(193, capturingFilterA.numRowsProcessed()); + assertEquals(14729, result.size()); + + // Use the unwrapped filter to test other optimization paths (i.e. chunk filtering) and assert equality. + assertTableEquals(result, memTable.where(Filter.and(filterSym, filterA))); + assertTableEquals(result, diskTable.where(Filter.and(filterSym, filterA))); + + allFilters.forEach(RowSetCapturingFilter::reset); + } + } } diff --git a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/location/ParquetTableLocationTest.java b/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/location/ParquetTableLocationTest.java deleted file mode 100644 index 4ea61be8402..00000000000 --- a/extensions/parquet/table/src/test/java/io/deephaven/parquet/table/location/ParquetTableLocationTest.java +++ /dev/null @@ -1,19 +0,0 @@ -// -// Copyright (c) 2016-2026 Deephaven Data Labs and Patent Pending -// -package io.deephaven.parquet.table.location; - - -import org.junit.Test; - -import static org.assertj.core.api.Assertions.assertThat; - -public class ParquetTableLocationTest { - @Test - public void testCostSortedValues() { - assertThat(ParquetTableLocation.PushdownMode.costSortedValues() - .stream() - .mapToLong(ParquetTableLocation.PushdownMode::filterCost)) - .isSorted(); - } -}