From 94a5b816aa9ab3d147111c04dd3dce1ee7e69e39 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 6 Dec 2021 22:41:01 -0500 Subject: [PATCH 01/44] Address a typo in JavaDoc. --- .../main/java/io/deephaven/api/agg/AggregationOptimizer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java index 66e1a2e5bc7..5ab505315d7 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java @@ -10,9 +10,10 @@ import java.util.Map.Entry; /** - * Optimizes a collection of {@link Aggregation aggregations} by grouping like-speccedd aggregations together. + * Optimizes a collection of {@link Aggregation aggregations} by grouping like-specced aggregations together. */ public final class AggregationOptimizer implements Aggregation.Visitor { + private static final Object COUNT_OBJ = new Object(); /** From 7fd696a656483516b49b2101fd3ffce6a3ccd9c7 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 8 Dec 2021 10:16:42 -0500 Subject: [PATCH 02/44] WIP on cleanup --- .../table/impl/by/AggregationFactory.java | 495 +----------------- .../table/impl/by/AggregationProcessor.java | 162 ++++++ 2 files changed, 165 insertions(+), 492 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java index a4032a59d59..952b05fe01b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java @@ -31,33 +31,8 @@ import java.util.stream.Stream; /** - * The AggregationFactory combines one or more aggregations into an {@link AggregationSpec} for use internally by the - * implementation of {@link Table#aggBy}. - * - *

- * The intended use of this class is to call the {@link #AggCombo(AggregationElement...)} method with a set of - * aggregations defined by: - *

+ * The AggregationProcessor combines one or more {@link Aggregation aggregations} into an + * {@link AggregationContextFactory} for use internally by the implementation of {@link Table#aggBy}. */ public class AggregationFactory implements AggregationSpec { static final String ROLLUP_RUNNING_SUM_COLUMN_ID = "_RS_"; @@ -74,470 +49,6 @@ public class AggregationFactory implements AggregationSpec { public static final String ROLLUP_COLUMN_SUFFIX = "__ROLLUP__"; - /** - * Create a new AggregationFactory suitable for passing to - * {@link QueryTable#by(AggregationSpec, io.deephaven.engine.table.impl.select.SelectColumn...)}. - * - * @param aggregations the aggregations to compute - * - * @return a new table with the specified aggregations. - */ - public static AggregationFactory AggCombo(AggregationElement... aggregations) { - return new AggregationFactory(aggregations); - } - - /** - * Create a formula aggregation. - * - * @param formula the formula to apply to each group - * @param formulaParam the parameter name within the formula - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggFormula(String formula, String formulaParam, final String... matchPairs) { - return new AggregationElementImpl(new AggregationFormulaSpec(formula, formulaParam), matchPairs); - } - - /** - * Create a minimum aggregation, equivalent to {@link Table#minBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggMin(final String... matchPairs) { - return Agg(AggType.Min, matchPairs); - } - - /** - * Create a maximum aggregation, equivalent to {@link Table#maxBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggMax(final String... matchPairs) { - return Agg(AggType.Max, matchPairs); - } - - /** - * Create a summation aggregation, equivalent to {@link Table#sumBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggSum(final String... matchPairs) { - return Agg(AggType.Sum, matchPairs); - } - - /** - * Create an absolute sum aggregation, equivalent to {@link Table#absSumBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggAbsSum(final String... matchPairs) { - return Agg(AggType.AbsSum, matchPairs); - } - - /** - * Create a variance aggregation, equivalent to {@link Table#varBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggVar(final String... matchPairs) { - return Agg(AggType.Var, matchPairs); - } - - /** - * Create an average aggregation, equivalent to {@link Table#avgBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggAvg(final String... matchPairs) { - return Agg(AggType.Avg, matchPairs); - } - - /** - * Create a weighted average aggregation, equivalent to {@link Table#wavgBy(String, String...)}. - * - * @param weight the name of the column to use as the weight for the average - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggWAvg(final String weight, final String... matchPairs) { - return Agg(new WeightedAverageSpecImpl(weight), matchPairs); - } - - /** - * Create a weighted sum aggregation, equivalent to {@link Table#wsumBy(String, String...)}. - * - * @param weight the name of the column to use as the weight for the sum - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggWSum(final String weight, final String... matchPairs) { - return Agg(new WeightedSumSpecImpl(weight), matchPairs); - } - - /** - * Create a median aggregation, equivalent to {@link Table#medianBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggMed(final String... matchPairs) { - return AggPct(0.50d, true, matchPairs); - } - - /** - * Create a standard deviation aggregation, equivalent to {@link Table#stdBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggStd(final String... matchPairs) { - return Agg(AggType.Std, matchPairs); - } - - /** - * Create a first aggregation, equivalent to {@link Table#firstBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggFirst(final String... matchPairs) { - return Agg(AggType.First, matchPairs); - } - - /** - * Create a last aggregation, equivalent to {@link Table#lastBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggLast(final String... matchPairs) { - return Agg(AggType.Last, matchPairs); - } - - /** - * Create a sorted first aggregation, equivalent to {@link io.deephaven.engine.util.SortedBy#sortedFirstBy}. - * - * @param sortColumn the column to sort by - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggSortedFirst(final String sortColumn, final String... matchPairs) { - return Agg(new SortedFirstBy(sortColumn), matchPairs); - } - - /** - * Create a sorted last aggregation, equivalent to {@link io.deephaven.engine.util.SortedBy#sortedLastBy}. - * - * @param sortColumn the column to sort by - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggSortedLast(final String sortColumn, final String... matchPairs) { - return Agg(new SortedLastBy(sortColumn), matchPairs); - } - - /** - * Create a sorted first aggregation, equivalent to {@link io.deephaven.engine.util.SortedBy#sortedFirstBy}. - * - * @param sortColumns the column to sort by - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggSortedFirst(final String[] sortColumns, final String... matchPairs) { - return Agg(new SortedFirstBy(sortColumns), matchPairs); - } - - /** - * Create a sorted last aggregation, equivalent to {@link io.deephaven.engine.util.SortedBy#sortedLastBy}. - * - * @param sortColumns the columns to sort by - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggSortedLast(final String[] sortColumns, final String... matchPairs) { - return Agg(new SortedLastBy(sortColumns), matchPairs); - } - - /** - * Create a group aggregation, equivalent to {@link Table#groupBy(String...)}. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggGroup(final String... matchPairs) { - return Agg(AggType.Group, matchPairs); - } - - /** - * Create an count aggregation, equivalent to {@link Table#countBy(String)}. - * - * @param resultColumn the name of the result column containing the count of each group - * - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggCount(final String resultColumn) { - return new CountAggregationElement(resultColumn); - } - - /** - * Create a distinct count aggregation. - * - * The output column contains the number of distinct values for the input column in that group. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)}. Null values - * are not counted. - */ - public static AggregationElement AggCountDistinct(final String... matchPairs) { - return AggCountDistinct(false, matchPairs); - } - - /** - * Create a distinct count aggregation. - * - * The output column contains the number of distinct values for the input column in that group. - * - * @param countNulls if true null values are counted as a distinct value, otherwise null values are ignored - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggCountDistinct(boolean countNulls, final String... matchPairs) { - return Agg(new CountDistinctSpec(countNulls), matchPairs); - } - - /** - * Create a distinct aggregation. - * - * The output column contains a {@link Vector} with the distinct values for the input column within the group. - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)}. Null values - * are ignored. - */ - public static AggregationElement AggDistinct(final String... matchPairs) { - return AggDistinct(false, matchPairs); - } - - - /** - * Create a distinct aggregation. - * - * The output column contains a {@link Vector} with the distinct values for the input column within the group. - * - * @param countNulls if true, then null values are included in the result, otherwise null values are ignored - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggDistinct(boolean countNulls, final String... matchPairs) { - return Agg(new DistinctSpec(countNulls), matchPairs); - } - - /** - * Create a Unique aggregation. - * - * The output column contains a value of the same type as the input column which contains
- * - * - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggUnique(final String... matchPairs) { - return Agg(new UniqueSpec(false), matchPairs); - } - - /** - * Create a Unique aggregation. - * - * The output column contains a value of the same type as the input column which contains
- * - * - * @param countNulls if true, then null values are included in the result, otherwise null values are ignored - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)}. Output - * columns contain null if there are no values present or there are more than 1 distinct values present. - */ - public static AggregationElement AggUnique(boolean countNulls, final String... matchPairs) { - return AggUnique(countNulls, null, null, matchPairs); - } - - /** - * Create a Unique aggregation. - * - * The output column contains a value of the same type as the input column which contains
- * - * - * @param countNulls if true, then null values are included in the result, otherwise null values are ignored - * @param noKeyValue the value to use if there are no values present - * @param nonUniqueValue the value to use if there are more than 1 values present - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggUnique(boolean countNulls, Object noKeyValue, Object nonUniqueValue, - final String... matchPairs) { - return Agg(new UniqueSpec(countNulls, noKeyValue, nonUniqueValue), matchPairs); - } - - /** - * Create a percentile aggregation. - * - * @param percentile the percentile to calculate - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggPct(double percentile, final String... matchPairs) { - return Agg(new PercentileBySpecImpl(percentile), matchPairs); - } - - /** - * Create a percentile aggregation. - * - * @param percentile the percentile to calculate - * @param averageMedian if true, then when the upper values and lower values have an equal size; average the highest - * lower value and lowest upper value to produce the median value for integers, longs, doubles, and floats - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement AggPct(double percentile, boolean averageMedian, final String... matchPairs) { - return Agg(new PercentileBySpecImpl(percentile, averageMedian), matchPairs); - } - - /** - * Create an aggregation. - * - * @param factory aggregation factory. - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement Agg(AggregationSpec factory, final String... matchPairs) { - return new AggregationElementImpl(factory, matchPairs); - } - - /** - * Create an aggregation. - * - * @param factory aggregation factory. - * @param matchPairs the columns to apply the aggregation to. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement Agg(AggregationSpec factory, final MatchPair... matchPairs) { - return new AggregationElementImpl(factory, matchPairs); - } - - /** - * Create an aggregation. - * - * @param factoryType aggregation factory type. - * @param matchPairs the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - * the same name, then the column name can be specified. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement Agg(AggType factoryType, final String... matchPairs) { - return Agg(factoryType, MatchPairFactory.getExpressions(matchPairs)); - } - - /** - * Create an aggregation. - * - * @param factoryType aggregation factory type. - * @param matchPairs the columns to apply the aggregation to. - * @return a AggregationElement object suitable for passing to {@link #AggCombo(AggregationElement...)} - */ - public static AggregationElement Agg(AggType factoryType, final MatchPair... matchPairs) { - final AggregationSpec factory; - switch (factoryType) { - case Min: - factory = new MinMaxBySpecImpl(true); - break; - case Max: - factory = new MinMaxBySpecImpl(false); - break; - case Sum: - factory = new SumSpec(); - break; - case AbsSum: - factory = new AbsSumSpec(); - break; - case Avg: - factory = new AvgSpec(); - break; - case Var: - factory = new VarSpec(); - break; - case Std: - factory = new StdSpec(); - break; - case First: - factory = new FirstBySpecImpl(); - break; - case Last: - factory = new LastBySpecImpl(); - break; - case Group: - factory = new AggregationGroupSpec(); - break; - case CountDistinct: - factory = new CountDistinctSpec(); - break; - case Distinct: - factory = new DistinctSpec(); - break; - case Unique: - factory = new UniqueSpec(); - break; - case Skip: - throw new IllegalArgumentException("Skip is not a valid aggregation type for AggCombo."); - default: - throw new UnsupportedOperationException("Unknown AggType: " + factoryType); - } - return new AggregationElementImpl(factory, matchPairs); - } - /** * Create a factory for performing rollups. */ @@ -1489,6 +1000,7 @@ private static String makeRedirectionName(IterativeIndexSpec inputAggregationSta } private static class RollupTableMapAndReverseLookupAttributeSetter implements AggregationContextTransformer { + private final PartitionByChunkedOperator tableMapOperator; private final AggregationFactory factory; private final boolean secondLevel; @@ -1561,5 +1073,4 @@ private static void setLeafRollupAttributes(QueryTable table) { table.setAttribute(Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, EmptyTableMap.INSTANCE); table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, ReverseLookup.NULL); } - } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java new file mode 100644 index 00000000000..5afaa3c1ab6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -0,0 +1,162 @@ +package io.deephaven.engine.table.impl.by; + +import io.deephaven.api.ColumnName; +import io.deephaven.api.agg.*; +import io.deephaven.api.agg.spec.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.engine.table.ChunkSource; +import org.jetbrains.annotations.NotNull; + +import java.util.*; + +/** + * Conversion tool to generate an {@link AggregationContextFactory} for a collection of + * {@link Aggregation aggregations}. + */ +public class AggregationProcessor implements Aggregation.Visitor, AggSpec.Visitor { + + /** + * Convert a collection of {@link Aggregation aggregations} to an {@link AggregationContextFactory}. + * + * @param aggregations The {@link Aggregation aggregations} + * @return The {@link AggregationContextFactory} + */ + public static AggregationContextFactory of(@NotNull final Collection aggregations) { + return + } + + private final List operators = new ArrayList<>(); + private final List inputNames = new ArrayList<>(); + private final List> inputColumns = new ArrayList<>(); + private final List transformers = new ArrayList<>(); + + private AggregationContextFactory build() { + final IterativeChunkedAggregationOperator[] operatorsArray = operators + .toArray(IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY); + final String[][] inputNamesArray = inputNames + .toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); + // noinspection unchecked + final ChunkSource.WithPrev[] inputColumnsArray = inputColumns + .toArray(ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); + final AggregationContextTransformer[] transformersArray = transformers + .toArray(AggregationContextTransformer.ZERO_LENGTH_AGGREGATION_CONTEXT_TRANSFORMER_ARRAY); + + return new AggregationContext(operatorsArray, inputNamesArray, inputColumnsArray, transformersArray, true); + } + + @Override + public void visit(Count count) { + + } + + @Override + public void visit(NormalAggregation normalAgg) { + + } + + @Override + public void visit(NormalAggregations normalAggs) { + + } + + @Override + public void visit(AggSpecAbsSum absSum) { + + } + + @Override + public void visit(AggSpecCountDistinct countDistinct) { + + } + + @Override + public void visit(AggSpecDistinct distinct) { + + } + + @Override + public void visit(AggSpecGroup group) { + + } + + @Override + public void visit(AggSpecAvg avg) { + + } + + @Override + public void visit(AggSpecFirst first) { + + } + + @Override + public void visit(AggSpecFormula formula) { + + } + + @Override + public void visit(AggSpecLast last) { + + } + + @Override + public void visit(AggSpecMax max) { + + } + + @Override + public void visit(AggSpecMedian median) { + + } + + @Override + public void visit(AggSpecMin min) { + + } + + @Override + public void visit(AggSpecPercentile pct) { + + } + + @Override + public void visit(AggSpecSortedFirst sortedFirst) { + + } + + @Override + public void visit(AggSpecSortedLast sortedLast) { + + } + + @Override + public void visit(AggSpecStd std) { + + } + + @Override + public void visit(AggSpecSum sum) { + + } + + @Override + public void visit(AggSpecUnique unique) { + + } + + @Override + public void visit(AggSpecWAvg wAvg) { + + } + + @Override + public void visit(AggSpecWSum wSum) { + + } + + @Override + public void visit(AggSpecVar var) { + + } +} From 2bd23d44ef5477ed5bd7b2b885bd1785065331f1 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 9 Dec 2021 14:52:15 -0500 Subject: [PATCH 03/44] Add missing AggSpecUnique options. --- .../table/impl/by/AggregationSpecAdapter.java | 2 +- .../java/io/deephaven/api/agg/Aggregation.java | 4 ++++ .../io/deephaven/api/agg/spec/AggSpec.java | 4 ++++ .../deephaven/api/agg/spec/AggSpecUnique.java | 18 ++++++++++++++++++ 4 files changed, 27 insertions(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java index 5de0a35a830..def203b9ddd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java @@ -126,7 +126,7 @@ public void visit(AggSpecSum sum) { @Override public void visit(AggSpecUnique unique) { - out = new UniqueSpec(unique.includeNulls()); + out = new UniqueSpec(unique.includeNulls(), unique.onlyNullsValue(), unique.nonUniqueValue()); } @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 38fa3963af7..f30bceeee8f 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -132,6 +132,10 @@ static Aggregation AggUnique(boolean includeNulls, String... pairs) { return of(AggSpec.unique(includeNulls), pairs); } + static Aggregation AggUnique(boolean includeNulls, Object onlyNullsValue, Object nonUniqueValue, String... pairs) { + return of(AggSpec.unique(includeNulls, onlyNullsValue, nonUniqueValue), pairs); + } + static Aggregation AggVar(String... pairs) { return of(AggSpec.var(), pairs); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 085fed1b811..dd67d1cb4d1 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -122,6 +122,10 @@ static AggSpecUnique unique(boolean includeNulls) { return AggSpecUnique.of(includeNulls); } + static AggSpecUnique unique(boolean includeNulls, Object onlyNullsValue, Object nonUniqueValue) { + return AggSpecUnique.of(includeNulls, onlyNullsValue, nonUniqueValue); + } + static AggSpecVar var() { return AggSpecVar.of(); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java index 08bb12b3d57..8954801fc77 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java @@ -16,11 +16,29 @@ public static AggSpecUnique of(boolean includeNulls) { return ImmutableAggSpecUnique.builder().includeNulls(includeNulls).build(); } + public static AggSpecUnique of(boolean includeNulls, Object onlyNullsValue, Object nonUniqueValue) { + return ImmutableAggSpecUnique.builder() + .includeNulls(includeNulls) + .onlyNullsValue(onlyNullsValue) + .nonUniqueValue(nonUniqueValue) + .build(); + } + @Default public boolean includeNulls() { return false; } + @Default + public Object onlyNullsValue() { + return null; + } + + @Default + public Object nonUniqueValue() { + return null; + } + @Override public final V walk(V visitor) { visitor.visit(this); From 123306c35d1dae9c88f9985d3d38986c981e5f64 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 9 Dec 2021 19:24:32 -0500 Subject: [PATCH 04/44] First cut of AggregationProcessor. Complete but-for rollups and a few esoteric aggs not in the API yet. --- .../engine/table/impl/TupleSourceFactory.java | 4 +- .../table/impl/by/AggregationContext.java | 23 +- .../impl/by/AggregationContextFactory.java | 1 + .../table/impl/by/AggregationFactory.java | 19 +- .../table/impl/by/AggregationProcessor.java | 458 +++++++++++++----- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- .../by/ChunkedOperatorAggregationHelper.java | 4 - .../table/impl/by/IterativeOperatorSpec.java | 17 +- .../impl/by/KeyOnlyAggregationFactory.java | 3 +- .../by/MinMaxIterativeOperatorFactory.java | 2 +- .../PercentileIterativeOperatorFactory.java | 2 +- ...SortedFirstOrLastByAggregationFactory.java | 2 +- .../engine/table/impl/by/StdSpec.java | 2 +- .../engine/table/impl/by/UniqueSpec.java | 2 +- .../engine/table/impl/by/VarSpec.java | 2 +- .../DistinctOperatorFactory.java | 53 +- .../unique/CharRollupUniqueOperator.java | 8 +- .../impl/select/AbstractRangeFilter.java | 3 +- .../io/deephaven/engine/util/SortedBy.java | 61 +-- .../tuplesource/TupleSourceCreatorImpl.java | 11 +- 20 files changed, 415 insertions(+), 264 deletions(-) diff --git a/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java b/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java index 03343cca1cb..e32cb10ab48 100644 --- a/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java +++ b/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java @@ -30,7 +30,7 @@ public interface TupleSourceCreator { /** * See {@link TupleSourceFactory#makeTupleSource(ColumnSource[])}. */ - TupleSource makeTupleSource(@NotNull ColumnSource... columnSources); + TupleSource makeTupleSource(@NotNull ColumnSource... columnSources); } /** @@ -39,7 +39,7 @@ public interface TupleSourceCreator { * @param columnSources The column sources * @return The tuple factory */ - public static TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) { + public static TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) { return tupleSourceCreator().makeTupleSource(columnSources); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java index 4b965f188e6..3ab42822479 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java @@ -51,11 +51,6 @@ class AggregationContext { */ private final boolean requiresRunFinds; - /** - * True if slots that are removed and then reincarnated should be modified. - */ - private final boolean addedBackModified; - /** * Do any operators require inputs. */ @@ -82,22 +77,15 @@ class AggregationContext { AggregationContext(IterativeChunkedAggregationOperator[] operators, String[][] inputNames, ChunkSource.WithPrev[] inputColumns) { - this(operators, inputNames, inputColumns, true); - } - - AggregationContext(IterativeChunkedAggregationOperator[] operators, String[][] inputNames, - ChunkSource.WithPrev[] inputColumns, boolean addedBackModified) { - this(operators, inputNames, inputColumns, null, true); + this(operators, inputNames, inputColumns, null); } AggregationContext(IterativeChunkedAggregationOperator[] operators, String[][] inputNames, - ChunkSource.WithPrev[] inputColumns, AggregationContextTransformer[] transformers, - boolean addedBackModified) { + ChunkSource.WithPrev[] inputColumns, AggregationContextTransformer[] transformers) { this.operators = operators; this.inputNames = inputNames; this.inputColumns = inputColumns; this.transformers = transformers; - this.addedBackModified = addedBackModified; requiresIndices = Arrays.stream(this.operators).anyMatch(IterativeChunkedAggregationOperator::requiresRowKeys); requiresRunFinds = Arrays.stream(this.operators).anyMatch(IterativeChunkedAggregationOperator::requiresRunFinds); @@ -466,13 +454,6 @@ PermuteKernel[] makePermuteKernels() { return permuteKernels; } - /** - * Returns true if slots that are removed and then reincarnated on the same cycle should be marked as modified. - */ - boolean addedBackModified() { - return addedBackModified; - } - void setReverseLookupFunction(ToIntFunction reverseLookupFunction) { if (transformers == null) { return; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContextFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContextFactory.java index 00a007fdf93..1cbcb95e255 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContextFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContextFactory.java @@ -6,6 +6,7 @@ /** * Produces an AggregationContext for aggregations given a table and the names of the group by columns. */ +@FunctionalInterface public interface AggregationContextFactory { /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java index 952b05fe01b..a5aac6ae073 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java @@ -19,7 +19,6 @@ import io.deephaven.engine.table.impl.sources.SingleValueObjectColumnSource; import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; import io.deephaven.time.DateTime; -import io.deephaven.vector.Vector; import org.jetbrains.annotations.NotNull; import java.math.BigDecimal; @@ -768,10 +767,8 @@ public AggregationContextFactory makeAggregationContextFactory() { ((UniqueSpec) inputAggregationSpec).countNulls(); // noinspection ConstantConditions op = IterativeOperatorSpec.getUniqueChunked( - lastLevelResult.getType(), mp.leftColumn(), countNulls, true, - ((UniqueSpec) inputAggregationSpec).getNoKeyValue(), - ((UniqueSpec) inputAggregationSpec) - .getNonUniqueValue(), + lastLevelResult.getType(), mp.leftColumn(), countNulls, ((UniqueSpec) inputAggregationSpec).getNoKeyValue(), ((UniqueSpec) inputAggregationSpec) + .getNonUniqueValue(), true, true); } @@ -805,8 +802,8 @@ public AggregationContextFactory makeAggregationContextFactory() { ssmChunkedMinMaxOperator.makeSecondaryOperator(isMinimum, resultName)); hasSource = false; } else { - operators.add(IterativeOperatorSpec.getMinMaxChunked(type, isMinimum, - isStream || isAddOnly, resultName)); + operators.add(IterativeOperatorSpec.getMinMaxChunked(type, resultName, isMinimum, + isStream || isAddOnly)); hasSource = true; } } else if (isPercentile) { @@ -815,11 +812,11 @@ public AggregationContextFactory makeAggregationContextFactory() { "Percentile or Median can not be used in a rollup!"); } operators.add(IterativeOperatorSpec.getPercentileChunked(type, - ((PercentileBySpecImpl) inputAggregationSpec) + resultName, ((PercentileBySpecImpl) inputAggregationSpec) .getPercentile(), ((PercentileBySpecImpl) inputAggregationSpec) - .getAverageMedian(), - resultName)); + .getAverageMedian() + )); hasSource = true; } else { operators.add(((IterativeOperatorSpec) inputAggregationSpec) @@ -984,7 +981,7 @@ public AggregationContextFactory makeAggregationContextFactory() { final ChunkSource.WithPrev[] inputColumnsArray = inputColumns.toArray(ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); - return new AggregationContext(operatorsArray, inputNamesArray, inputColumnsArray, transformersArray, true); + return new AggregationContext(operatorsArray, inputNamesArray, inputColumnsArray, transformersArray); }; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 5afaa3c1ab6..a92adc05db6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1,20 +1,36 @@ package io.deephaven.engine.table.impl.by; -import io.deephaven.api.ColumnName; +import io.deephaven.api.SortColumn; import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.BaseTable; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.TupleSourceFactory; +import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; +import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.time.DateTime; import org.jetbrains.annotations.NotNull; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.function.BiFunction; +import java.util.stream.IntStream; + +import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; +import static io.deephaven.engine.table.impl.by.IterativeOperatorSpec.*; /** - * Conversion tool to generate an {@link AggregationContextFactory} for a collection of - * {@link Aggregation aggregations}. + * Conversion tool to generate an {@link AggregationContextFactory} for a collection of {@link Aggregation + * aggregations}. */ -public class AggregationProcessor implements Aggregation.Visitor, AggSpec.Visitor { +public class AggregationProcessor implements AggregationContextFactory { /** * Convert a collection of {@link Aggregation aggregations} to an {@link AggregationContextFactory}. @@ -23,140 +39,322 @@ public class AggregationProcessor implements Aggregation.Visitor, AggSpec.Visito * @return The {@link AggregationContextFactory} */ public static AggregationContextFactory of(@NotNull final Collection aggregations) { - return - } - - private final List operators = new ArrayList<>(); - private final List inputNames = new ArrayList<>(); - private final List> inputColumns = new ArrayList<>(); - private final List transformers = new ArrayList<>(); - - private AggregationContextFactory build() { - final IterativeChunkedAggregationOperator[] operatorsArray = operators - .toArray(IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY); - final String[][] inputNamesArray = inputNames - .toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); - // noinspection unchecked - final ChunkSource.WithPrev[] inputColumnsArray = inputColumns - .toArray(ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); - final AggregationContextTransformer[] transformersArray = transformers - .toArray(AggregationContextTransformer.ZERO_LENGTH_AGGREGATION_CONTEXT_TRANSFORMER_ARRAY); - - return new AggregationContext(operatorsArray, inputNamesArray, inputColumnsArray, transformersArray, true); - } - - @Override - public void visit(Count count) { - - } - - @Override - public void visit(NormalAggregation normalAgg) { - - } - - @Override - public void visit(NormalAggregations normalAggs) { - - } - - @Override - public void visit(AggSpecAbsSum absSum) { - - } - - @Override - public void visit(AggSpecCountDistinct countDistinct) { - - } - - @Override - public void visit(AggSpecDistinct distinct) { - - } - - @Override - public void visit(AggSpecGroup group) { - - } - - @Override - public void visit(AggSpecAvg avg) { - - } - - @Override - public void visit(AggSpecFirst first) { - - } - - @Override - public void visit(AggSpecFormula formula) { - + return new AggregationProcessor(aggregations); } - @Override - public void visit(AggSpecLast last) { - - } - - @Override - public void visit(AggSpecMax max) { - - } - - @Override - public void visit(AggSpecMedian median) { - - } - - @Override - public void visit(AggSpecMin min) { + private final Collection aggregations; + private AggregationProcessor(@NotNull final Collection aggregations) { + this.aggregations = aggregations; } - @Override - public void visit(AggSpecPercentile pct) { - - } + // ----------------------------------------------------------------------------------------------------------------- + // AggregationContextFactory + // ----------------------------------------------------------------------------------------------------------------- @Override - public void visit(AggSpecSortedFirst sortedFirst) { - + public AggregationContext makeAggregationContext(@NotNull Table table, @NotNull String... groupByColumnNames) { + return new Converter(table, groupByColumnNames).build(); } - @Override - public void visit(AggSpecSortedLast sortedLast) { - - } - - @Override - public void visit(AggSpecStd std) { - - } - - @Override - public void visit(AggSpecSum sum) { - - } - - @Override - public void visit(AggSpecUnique unique) { - - } - - @Override - public void visit(AggSpecWAvg wAvg) { - - } - - @Override - public void visit(AggSpecWSum wSum) { - + /** + * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an + * {@link AggregationContext}. Accumulates state by visiting each aggregation. + */ + private class Converter implements Aggregation.Visitor, AggSpec.Visitor { + + private final Table table; + private final String[] groupByColumnNames; + + private final boolean isAddOnly; + private final boolean isStream; + + private final List operators = new ArrayList<>(); + private final List inputColumnNames = new ArrayList<>(); + private final List> inputSources = new ArrayList<>(); + private final List transformers = new ArrayList<>(); + + private List resultPairs; + private int trackedFirstOrLastIndex = -1; + private boolean partitionFound = false; // TODO-RWC: Add rollup support + + private Converter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + this.table = table; + this.groupByColumnNames = groupByColumnNames; + isAddOnly = ((BaseTable) table).isAddOnly(); + isStream = ((BaseTable) table).isStream(); + } + + private AggregationContext build() { + for (final Aggregation aggregation : aggregations) { + aggregation.walk(this); + } + // noinspection unchecked + return new AggregationContext( + operators.toArray(IterativeChunkedAggregationOperator[]::new), + inputColumnNames.toArray(String[][]::new), + inputSources.toArray(ChunkSource.WithPrev[]::new), + transformers.toArray(AggregationContextTransformer[]::new)); + } + + // ------------------------------------------------------------------------------------------------------------- + // Aggregation.Visitor + // ------------------------------------------------------------------------------------------------------------- + + @Override + public void visit(@NotNull final Count count) { + operators.add(new CountAggregationOperator(count.column().name())); + inputSources.add(null); + inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); + } + + @Override + public void visit(@NotNull final NormalAggregation normalAgg) { + resultPairs = List.of(normalAgg.pair()); + normalAgg.spec().walk(this); + resultPairs = null; + } + + @Override + public void visit(@NotNull final NormalAggregations normalAggs) { + resultPairs = normalAggs.pairs(); + normalAggs.spec().walk(this); + resultPairs = null; + } + + // ------------------------------------------------------------------------------------------------------------- + // AggSpec.Visitor + // ------------------------------------------------------------------------------------------------------------- + + @Override + public void visit(@NotNull final AggSpecAbsSum absSum) { + // TODO-RWC: Move this helper and its friends here or to a new landing place + visitBasicAgg(IterativeOperatorSpec::getAbsSumChunked); + } + + @Override + public void visit(@NotNull final AggSpecCountDistinct countDistinct) { + visitBasicAgg((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), false, false)); + } + + @Override + public void visit(@NotNull final AggSpecDistinct distinct) { + visitBasicAgg((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), false, false)); + } + + @Override + public void visit(@NotNull final AggSpecGroup group) { + streamUnsupported("Group"); + operators.add(new GroupByChunkedOperator((QueryTable) table, true, MatchPair.fromPairs(resultPairs))); + inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); + inputSources.add(null); + } + + @Override + public void visit(@NotNull final AggSpecAvg avg) { + visitBasicAgg((t, n) -> getAvgChunked(t, n, false)); + } + + @Override + public void visit(@NotNull final AggSpecFirst first) { + visitFirstOrLastAgg(true, null); + } + + @Override + public void visit(@NotNull final AggSpecFormula formula) { + streamUnsupported("Formula"); + final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator((QueryTable) table, false, + resultPairs.stream().map(pair -> MatchPair.of((Pair) pair.input())).toArray(MatchPair[]::new)); + final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator(groupByChunkedOperator, + true, formula.formula(), formula.formulaParam(), MatchPair.fromPairs(resultPairs)); + operators.add(formulaChunkedOperator); + inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); + inputSources.add(null); + } + + @Override + public void visit(@NotNull final AggSpecLast last) { + visitFirstOrLastAgg(false, null); + } + + @Override + public void visit(@NotNull final AggSpecMax max) { + visitMinOrMaxAgg(false); + } + + @Override + public void visit(@NotNull final AggSpecMedian median) { + visitBasicAgg((t, n) -> getPercentileChunked(t, n, 0.50d, median.averageMedian())); + } + + @Override + public void visit(@NotNull final AggSpecMin min) { + visitMinOrMaxAgg(true); + } + + @Override + public void visit(@NotNull final AggSpecPercentile pct) { + visitBasicAgg((t, n) -> getPercentileChunked(t, n, pct.percentile(), pct.averageMedian())); + } + + @Override + public void visit(@NotNull final AggSpecSortedFirst sortedFirst) { + visitSortedFirstOrLastAgg(sortedFirst.columns(), true); + } + + @Override + public void visit(@NotNull final AggSpecSortedLast sortedLast) { + visitSortedFirstOrLastAgg(sortedLast.columns(), false); + } + + @Override + public void visit(@NotNull final AggSpecStd std) { + visitBasicAgg((t, n) -> getVarChunked(t, n, true, false)); + } + + @Override + public void visit(@NotNull final AggSpecSum sum) { + visitBasicAgg(IterativeOperatorSpec::getSumChunked); + } + + @Override + public void visit(@NotNull final AggSpecUnique unique) { + visitBasicAgg((t, n) -> getUniqueChunked(t, n, + unique.includeNulls(), unique.onlyNullsValue(), unique.nonUniqueValue(), false, false)); + } + + @Override + public void visit(@NotNull final AggSpecWAvg wAvg) { + // TODO-RWC: Move this helper here or to a new landing place + WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, wAvg.weight().name(), false, + MatchPair.fromPairs(resultPairs), operators, inputColumnNames, inputSources); + } + + @Override + public void visit(@NotNull final AggSpecWSum wSum) { + WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, wSum.weight().name(), true, + MatchPair.fromPairs(resultPairs), operators, inputColumnNames, inputSources); + } + + @Override + public void visit(@NotNull final AggSpecVar var) { + visitBasicAgg((t, n) -> getVarChunked(t, n, false, false)); + } + + private void visitBasicAgg(BiFunction, String, IterativeChunkedAggregationOperator> operatorFactory) { + for (final Pair pair : resultPairs) { + final String inputName = pair.input().name(); + final String resultName = pair.output().name(); + final ColumnSource rawInputSource = table.getColumnSource(inputName); + final Class type = rawInputSource.getType(); + final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + + operators.add(operatorFactory.apply(type, resultName)); + inputColumnNames.add(new String[] {inputName}); + inputSources.add(inputSource); + } + } + + private void visitMinOrMaxAgg(final boolean isMin) { + for (final Pair pair : resultPairs) { + final String inputName = pair.input().name(); + final String resultName = pair.output().name(); + final ColumnSource rawInputSource = table.getColumnSource(inputName); + final Class type = rawInputSource.getType(); + final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + + IntStream.range(0, inputSources.size()) + .filter(index -> (inputSources.get(index) == inputSource) + && (operators.get(index) instanceof SsmChunkedMinMaxOperator)) + .findFirst().ifPresentOrElse( + (final int priorMinMaxIndex) -> { + final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = + (SsmChunkedMinMaxOperator) operators.get(priorMinMaxIndex); + operators.add(ssmChunkedMinMaxOperator.makeSecondaryOperator(isMin, resultName)); + inputColumnNames.add(new String[] {inputName}); + inputSources.add(null); + }, + () -> { + operators.add(getMinMaxChunked(type, resultName, isMin, isAddOnly || isStream)); + inputColumnNames.add(new String[] {inputName}); + inputSources.add(inputSource); + }); + } + } + + private void visitFirstOrLastAgg(final boolean isFirst, final String exposeRedirectionAs) { + if (exposeRedirectionAs != null) { + streamUnsupported((isFirst ? "First" : "Last") + " with exposed row redirections (e.g. for rollup())"); + } + final MatchPair[] resultMatchPairs = MatchPair.fromPairs(resultPairs); + if (table.isRefreshing()) { + if (isAddOnly) { + operators.add(new AddOnlyFirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, + exposeRedirectionAs)); + } else if (isStream) { + operators.add(isFirst + ? new StreamFirstChunkedOperator(resultMatchPairs, table) + : new StreamLastChunkedOperator(resultMatchPairs, table)); + } else { + if (trackedFirstOrLastIndex >= 0) { + operators.add(((FirstOrLastChunkedOperator) operators.get(trackedFirstOrLastIndex)) + .makeSecondaryOperator(isFirst, resultMatchPairs, table, exposeRedirectionAs)); + } else { + trackedFirstOrLastIndex = operators.size(); + operators.add(new FirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, + exposeRedirectionAs)); + } + } + } else { + operators.add(new StaticFirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, + exposeRedirectionAs)); + } + inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); + inputSources.add(null); + } + + private void streamUnsupported(@NotNull final String operationName) { + if (!isStream) { + return; + } + throw new UnsupportedOperationException(String.format( + "Stream tables do not support Agg%s; use StreamTableTools.streamToAppendOnlyTable to accumulate full history", + operationName)); + } + + private void visitSortedFirstOrLastAgg(@NotNull final List sortColumns, final boolean isFirst) { + final String[] sortColumnNames = sortColumns.stream().map(sc -> { + descendingSortedFirstOrLastUnsupported(sc, isFirst); + return sc.column().name(); + }).toArray(String[]::new); + final ChunkSource.WithPrev inputSource; + if (sortColumnNames.length == 1) { + inputSource = table.getColumnSource(sortColumnNames[0]); + } else { + // Create a tuple source, because our underlying SSA does not handle multiple sort columns + inputSource = TupleSourceFactory.makeTupleSource( + Arrays.stream(sortColumnNames).map(table::getColumnSource).toArray(ColumnSource[]::new)); + } + // TODO-RWC: Move this helper here or to a new landing place + operators.add(SortedFirstOrLastByAggregationFactory.makeOperator(inputSource.getChunkType(), isFirst, + aggregations.size() > 1, MatchPair.fromPairs(resultPairs), table)); + inputColumnNames.add(sortColumnNames); + inputSources.add(inputSource); + } + + private void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sortColumn, + final boolean isFirst) { + if (sortColumn.order() == SortColumn.Order.ASCENDING) { + return; + } + throw new UnsupportedOperationException(String.format("%s does not support sort order in %s", + isFirst ? "SortedFirst" : "SortedLast", sortColumn)); + } } - @Override - public void visit(AggSpecVar var) { - + private static ColumnSource maybeReinterpretDateTimeAsLong(@NotNull final ColumnSource inputSource) { + return inputSource.getType() == DateTime.class + ? ReinterpretUtils.dateTimeToLongSource(inputSource) + : inputSource; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.java index 061be9c4cb0..ef4a1290c1c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.java @@ -139,7 +139,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java index 84a8b043bfb..21648d342c8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java @@ -6,7 +6,6 @@ import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; -import io.deephaven.configuration.Configuration; import io.deephaven.datastructures.util.SmartKey; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; @@ -580,9 +579,6 @@ private TableUpdate computeDownstreamIndicesAndCopyKeys( downstream.modified = modifiedStatesBuilder.build(); downstream.modified().writableCast().remove(downstream.added()); downstream.modified().writableCast().remove(downstream.removed()); - if (ac.addedBackModified()) { - downstream.modified().writableCast().insert(addedBack); - } } ac.propagateChangesToOperators(downstream, newStates); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java index 07793bd8f84..4c2053afbb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java @@ -49,8 +49,8 @@ static IterativeChunkedAggregationOperator getSumChunked(Class type, String name throw new UnsupportedOperationException("Unsupported type " + type); } - static IterativeChunkedAggregationOperator getMinMaxChunked(Class type, boolean minimum, boolean isStreamOrAddOnly, - String name) { + static IterativeChunkedAggregationOperator getMinMaxChunked(Class type, String name, boolean minimum, + boolean isStreamOrAddOnly) { if (!isStreamOrAddOnly) { return new SsmChunkedMinMaxOperator(type, minimum, name); } else { @@ -76,8 +76,8 @@ static IterativeChunkedAggregationOperator getMinMaxChunked(Class type, boolean } } - static IterativeChunkedAggregationOperator getPercentileChunked(Class type, double percentile, - boolean averageMedian, String name) { + static IterativeChunkedAggregationOperator getPercentileChunked(Class type, String name, double percentile, + boolean averageMedian) { return new SsmChunkedPercentileOperator(type, percentile, averageMedian, name); } @@ -92,8 +92,9 @@ static IterativeChunkedAggregationOperator getDistinctChunked(Class type, String } static IterativeChunkedAggregationOperator getUniqueChunked(Class type, String name, boolean countNulls, - boolean exposeInternal, Object noKeyValue, Object nonUniqueValue, boolean isRollup) { - return DistinctOperatorFactory.createUnique(type, name, countNulls, exposeInternal, noKeyValue, nonUniqueValue, + Object noKeyValue, Object nonUniqueValue, + boolean exposeInternal, boolean isRollup) { + return DistinctOperatorFactory.createUnique(type, name, countNulls, noKeyValue, nonUniqueValue, exposeInternal, isRollup); } @@ -149,8 +150,8 @@ static IterativeChunkedAggregationOperator getAvgChunked(Class type, String name throw new UnsupportedOperationException("Unsupported type " + type); } - static IterativeChunkedAggregationOperator getVarChunked(Class type, boolean std, String name, - boolean exposeInternalColumns) { + static IterativeChunkedAggregationOperator getVarChunked(Class type, String name, boolean std, + boolean exposeInternalColumns) { if (type == Byte.class || type == byte.class) { return new ByteChunkedVarOperator(std, name, exposeInternalColumns); } else if (type == Character.class || type == char.class) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java index e0744c38016..5bbc9bafe08 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java @@ -16,7 +16,6 @@ public AggregationContext makeAggregationContext(@NotNull final Table table, return new AggregationContext( IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY, CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY, - ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY, - false); + ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java index fd9e24eed1b..67725feb195 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java @@ -21,7 +21,7 @@ public MinMaxIterativeOperatorFactory(boolean minimum, boolean isAddOnly) { @Override public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, boolean exposeInternalColumns) { - return getMinMaxChunked(type, minimum, isAddOnly, name); + return getMinMaxChunked(type, name, minimum, isAddOnly); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java index d015dfdefa2..efc0b1013e6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java @@ -19,7 +19,7 @@ public PercentileIterativeOperatorFactory(double percentile, boolean averageMedi @Override public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, boolean exposeInternalColumns) { - return IterativeOperatorSpec.getPercentileChunked(type, percentile, averageMedian, name); + return IterativeOperatorSpec.getPercentileChunked(type, name, percentile, averageMedian); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java index 060fbebbfec..91eb2e67665 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java @@ -73,7 +73,7 @@ public String toString() { return (isFirst ? "SortedFirstBy" : "SortedLastBy") + Arrays.toString(sortColumns); } - private static IterativeChunkedAggregationOperator makeOperator(@NotNull final ChunkType chunkType, + static IterativeChunkedAggregationOperator makeOperator(@NotNull final ChunkType chunkType, final boolean isFirst, final boolean isCombo, @NotNull final MatchPair[] resultPairs, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java index ad9abe4115b..e9fde99526d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java @@ -13,7 +13,7 @@ public StdSpec() {} @Override public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, boolean exposeInternalColumns) { - return getVarChunked(type, true, name, exposeInternalColumns); + return getVarChunked(type, name, true, exposeInternalColumns); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java index b22dc2b5855..67cd7804094 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java @@ -90,7 +90,7 @@ UniqueSpec rollupFactory() { @Override public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, boolean exposeInternalColumns) { - return getUniqueChunked(type, name, countNulls, exposeInternalColumns, noKeyValue, nonUniqueValue, + return getUniqueChunked(type, name, countNulls, noKeyValue, nonUniqueValue, exposeInternalColumns, secondRollup); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java index 0000f9e7870..2d2eadd3106 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java @@ -13,7 +13,7 @@ public VarSpec() {} @Override public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, boolean exposeInternalColumns) { - return getVarChunked(type, false, name, exposeInternalColumns); + return getVarChunked(type, name, false, exposeInternalColumns); } private static final AggregationMemoKey VAR_INSTANCE = new AggregationMemoKey() {}; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java index b6ea16a3378..79c6794f2af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java @@ -120,59 +120,60 @@ static IterativeChunkedAggregationOperator createDistinct(Class type, String * @return an appropriate operator. */ static IterativeChunkedAggregationOperator createUnique(Class type, String resultName, boolean countNulls, - boolean exposeInternal, Object noKeyValue, Object nonUniqueValue, boolean isRollup) { - checkType(resultName, "No Key Value", type, noKeyValue); + Object onlyNullsValue, Object nonUniqueValue, + boolean exposeInternal, boolean isRollup) { + checkType(resultName, "Only Nulls Value", type, onlyNullsValue); checkType(resultName, "Non Unique Value", type, nonUniqueValue); if (type == Byte.class || type == byte.class) { - final byte nkvAsType = (noKeyValue == null) ? NULL_BYTE : ((Number) noKeyValue).byteValue(); + final byte onvAsType = (onlyNullsValue == null) ? NULL_BYTE : ((Number) onlyNullsValue).byteValue(); final byte nuvAsType = (nonUniqueValue == null) ? NULL_BYTE : ((Number) nonUniqueValue).byteValue(); - return isRollup ? new ByteRollupUniqueOperator(resultName, countNulls, nkvAsType, nuvAsType) - : new ByteChunkedUniqueOperator(resultName, countNulls, exposeInternal, nkvAsType, nuvAsType); + return isRollup ? new ByteRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) + : new ByteChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); } else if (type == Character.class || type == char.class) { return isRollup ? new CharRollupUniqueOperator(resultName, countNulls, - io.deephaven.util.type.TypeUtils.unbox((Character) noKeyValue), + io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsValue), io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueValue)) : new CharChunkedUniqueOperator(resultName, countNulls, exposeInternal, - io.deephaven.util.type.TypeUtils.unbox((Character) noKeyValue), + io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsValue), io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueValue)); } else if (type == Double.class || type == double.class) { - final double nkvAsType = (noKeyValue == null) ? NULL_DOUBLE : ((Number) noKeyValue).doubleValue(); + final double onvAsType = (onlyNullsValue == null) ? NULL_DOUBLE : ((Number) onlyNullsValue).doubleValue(); final double nuvAsType = (nonUniqueValue == null) ? NULL_DOUBLE : ((Number) nonUniqueValue).doubleValue(); - return isRollup ? new DoubleRollupUniqueOperator(resultName, countNulls, nkvAsType, nuvAsType) - : new DoubleChunkedUniqueOperator(resultName, countNulls, exposeInternal, nkvAsType, nuvAsType); + return isRollup ? new DoubleRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) + : new DoubleChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); } else if (type == Float.class || type == float.class) { - final float nkvAsType = (noKeyValue == null) ? NULL_FLOAT : ((Number) noKeyValue).floatValue(); + final float onvAsType = (onlyNullsValue == null) ? NULL_FLOAT : ((Number) onlyNullsValue).floatValue(); final float nuvAsType = (nonUniqueValue == null) ? NULL_FLOAT : ((Number) nonUniqueValue).floatValue(); - return isRollup ? new FloatRollupUniqueOperator(resultName, countNulls, nkvAsType, nuvAsType) - : new FloatChunkedUniqueOperator(resultName, countNulls, exposeInternal, nkvAsType, nuvAsType); + return isRollup ? new FloatRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) + : new FloatChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); } else if (type == Integer.class || type == int.class) { - final int nkvAsType = (noKeyValue == null) ? NULL_INT : ((Number) noKeyValue).intValue(); + final int onvAsType = (onlyNullsValue == null) ? NULL_INT : ((Number) onlyNullsValue).intValue(); final int nuvAsType = (nonUniqueValue == null) ? NULL_INT : ((Number) nonUniqueValue).intValue(); - return isRollup ? new IntRollupUniqueOperator(resultName, countNulls, nkvAsType, nuvAsType) - : new IntChunkedUniqueOperator(resultName, countNulls, exposeInternal, nkvAsType, nuvAsType); + return isRollup ? new IntRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) + : new IntChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); } else if (type == Long.class || type == long.class || type == DateTime.class) { - final long nkvAsType; + final long onvAsType; final long nuvAsType; if (type == DateTime.class) { - nkvAsType = (noKeyValue == null) ? NULL_LONG : ((DateTime) noKeyValue).getNanos(); + onvAsType = (onlyNullsValue == null) ? NULL_LONG : ((DateTime) onlyNullsValue).getNanos(); nuvAsType = (nonUniqueValue == null) ? NULL_LONG : ((DateTime) nonUniqueValue).getNanos(); } else { - nkvAsType = (noKeyValue == null) ? NULL_LONG : ((Number) noKeyValue).longValue(); + onvAsType = (onlyNullsValue == null) ? NULL_LONG : ((Number) onlyNullsValue).longValue(); nuvAsType = (nonUniqueValue == null) ? NULL_LONG : ((Number) nonUniqueValue).longValue(); } - return isRollup ? new LongRollupUniqueOperator(type, resultName, countNulls, nkvAsType, nuvAsType) - : new LongChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, nkvAsType, nuvAsType); + return isRollup ? new LongRollupUniqueOperator(type, resultName, countNulls, onvAsType, nuvAsType) + : new LongChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onvAsType, nuvAsType); } else if (type == Short.class || type == short.class) { - final short nkvAsType = (noKeyValue == null) ? NULL_SHORT : ((Number) noKeyValue).shortValue(); + final short onvAsType = (onlyNullsValue == null) ? NULL_SHORT : ((Number) onlyNullsValue).shortValue(); final short nuvAsType = (nonUniqueValue == null) ? NULL_SHORT : ((Number) nonUniqueValue).shortValue(); - return isRollup ? new ShortRollupUniqueOperator(resultName, countNulls, nkvAsType, nuvAsType) - : new ShortChunkedUniqueOperator(resultName, countNulls, exposeInternal, nkvAsType, nuvAsType); + return isRollup ? new ShortRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) + : new ShortChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); } else { - return isRollup ? new ObjectRollupUniqueOperator(type, resultName, countNulls, noKeyValue, nonUniqueValue) - : new ObjectChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, noKeyValue, + return isRollup ? new ObjectRollupUniqueOperator(type, resultName, countNulls, onlyNullsValue, nonUniqueValue) + : new ObjectChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onlyNullsValue, nonUniqueValue); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java index c54eef7c43c..5d1082e5ebf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java @@ -45,7 +45,7 @@ public class CharRollupUniqueOperator implements IterativeChunkedAggregationOper private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final char noValueKey; + private final char onlyNullsValue; private final char nonUniqueKey; private UpdateCommitter prevFlusher = null; @@ -56,12 +56,12 @@ public CharRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - char noValueKey, + char onlyNullsValue, char nonUniqueKey) { this.name = name; this.countNull = countNulls; this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.onlyNullsValue = onlyNullsValue; // region SsmCreation this.ssms = new CharSsmBackedSource(); // endregion SsmCreation @@ -541,7 +541,7 @@ private static void flushPrevious(CharRollupUniqueOperator op) { //region Private Helpers private void updateResult(CharSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsValue); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java index 5c569b84687..8dce82a4f82 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java @@ -22,7 +22,8 @@ public abstract class AbstractRangeFilter extends WhereFilterImpl { private static final Pattern decimalPattern = Pattern.compile("(-)?\\d+(?:\\.((\\d+)0*)?)?"); - protected final String columnName; + protected final String + columnName; protected final boolean upperInclusive; protected final boolean lowerInclusive; diff --git a/engine/table/src/main/java/io/deephaven/engine/util/SortedBy.java b/engine/table/src/main/java/io/deephaven/engine/util/SortedBy.java index 460a5c852c8..a98b980dfc5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/util/SortedBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/util/SortedBy.java @@ -1,14 +1,13 @@ package io.deephaven.engine.util; -import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.table.Table; import io.deephaven.util.annotations.ScriptApi; import org.jetbrains.annotations.NotNull; -import java.util.*; +import java.util.Collection; -import static io.deephaven.api.agg.Aggregation.AggSortedFirst; -import static io.deephaven.api.agg.Aggregation.AggSortedLast; +import static io.deephaven.api.agg.spec.AggSpec.sortedFirst; +import static io.deephaven.api.agg.spec.AggSpec.sortedLast; /** * SortedBy operations sort the values in each of the buckets according to a specified column. The sortedFirstBy returns @@ -21,19 +20,6 @@ public class SortedBy { */ private SortedBy() {} - private static String[] resultColumns(@NotNull final Table input) { - return resultColumns(input, Collections.emptyList()); - } - - private static String[] resultColumns(@NotNull final Table input, @NotNull Collection groupByColumns) { - if (groupByColumns.isEmpty()) { - return input.getDefinition().getColumnNamesArray(); - } - final Set groupBySet = new HashSet<>(groupByColumns); - return input.getDefinition().getColumnNames().stream() - .filter(col -> !groupBySet.contains(col)).toArray(String[]::new); - } - /** * Return a new table with a single row, containing the lowest value of sortColumnName. * @@ -44,7 +30,7 @@ private static String[] resultColumns(@NotNull final Table input, @NotNull Colle */ @NotNull public static Table sortedFirstBy(@NotNull Table input, @NotNull String sortColumnName) { - return input.aggBy(AggSortedFirst(sortColumnName, resultColumns(input))); + return input.aggAllBy(sortedFirst(sortColumnName)); } /** @@ -57,7 +43,7 @@ public static Table sortedFirstBy(@NotNull Table input, @NotNull String sortColu */ @NotNull public static Table sortedFirstBy(@NotNull Table input, @NotNull String[] sortColumnNames) { - return input.aggBy(AggSortedFirst(Arrays.asList(sortColumnNames), resultColumns(input))); + return input.aggAllBy(sortedFirst(sortColumnNames)); } /** @@ -72,9 +58,7 @@ public static Table sortedFirstBy(@NotNull Table input, @NotNull String[] sortCo @NotNull public static Table sortedFirstBy(@NotNull Table input, @NotNull String sortColumnName, @NotNull String... groupByColumns) { - return input.aggBy( - AggSortedFirst(sortColumnName, resultColumns(input, Arrays.asList(groupByColumns))), - groupByColumns); + return input.aggAllBy(sortedFirst(sortColumnName), groupByColumns); } /** @@ -89,9 +73,7 @@ public static Table sortedFirstBy(@NotNull Table input, @NotNull String sortColu @NotNull public static Table sortedFirstBy(@NotNull Table input, @NotNull String[] sortColumnNames, @NotNull String... groupByColumns) { - return input.aggBy( - AggSortedFirst(Arrays.asList(sortColumnNames), resultColumns(input, Arrays.asList(groupByColumns))), - groupByColumns); + return input.aggAllBy(sortedFirst(sortColumnNames), groupByColumns); } /** @@ -106,9 +88,7 @@ public static Table sortedFirstBy(@NotNull Table input, @NotNull String[] sortCo @NotNull public static Table sortedFirstBy(@NotNull Table input, @NotNull String sortColumnName, @NotNull Collection groupByColumns) { - return input.aggBy( - AggSortedFirst(sortColumnName, resultColumns(input, groupByColumns)), - groupByColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); + return input.aggAllBy(sortedFirst(sortColumnName), groupByColumns); } /** @@ -123,9 +103,7 @@ public static Table sortedFirstBy(@NotNull Table input, @NotNull String sortColu @NotNull public static Table sortedFirstBy(@NotNull Table input, @NotNull Collection sortColumnNames, @NotNull Collection groupByColumns) { - return input.aggBy( - AggSortedFirst(sortColumnNames, resultColumns(input, groupByColumns)), - groupByColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); + return input.aggAllBy(sortedFirst(sortColumnNames), groupByColumns); } /** @@ -138,7 +116,7 @@ public static Table sortedFirstBy(@NotNull Table input, @NotNull Collection groupByColumns) { - return input.aggBy( - AggSortedLast(sortColumnName, resultColumns(input, groupByColumns)), - groupByColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); + return input.aggAllBy(sortedLast(sortColumnName), groupByColumns); } /** @@ -217,8 +190,6 @@ public static Table sortedLastBy(@NotNull Table input, @NotNull String sortColum @NotNull public static Table sortedLastBy(@NotNull Table input, @NotNull Collection sortColumnNames, @NotNull Collection groupByColumns) { - return input.aggBy( - AggSortedLast(sortColumnNames, resultColumns(input, groupByColumns)), - groupByColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); + return input.aggAllBy(sortedLast(sortColumnNames), groupByColumns); } } diff --git a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.java b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.java index 3dbd092dbc6..6f45b1920e0 100644 --- a/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.java +++ b/engine/tuplesource/src/main/java/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.java @@ -35,14 +35,16 @@ public TupleSourceFactory.TupleSourceCreator get() { private TupleSourceCreatorImpl() {} @Override - public TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) { + public TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) { final int length = columnSources.length; if (length == 0) { - return EmptyTupleSource.INSTANCE; + // noinspection unchecked + return (TupleSource) EmptyTupleSource.INSTANCE; } if (length == 1) { // NB: Don't reinterpret here, or you may have a bad time with join states when the LHS and RHS columns are // differently reinterpretable. + // noinspection unchecked return columnSources[0]; } if (length < 4) { @@ -74,6 +76,7 @@ public TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) e); } try { + // noinspection unchecked return factoryConstructor.newInstance((Object[]) Stream.of(typesNamesAndInternalSources) .map(Pair::getSecond).toArray(ColumnSource[]::new)); } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) { @@ -82,7 +85,9 @@ public TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) } // NB: Don't reinterpret here, or you may have a bad time with join states when the LHS and RHS columns are // differently reinterpretable. - return new MultiColumnTupleSource(Arrays.stream(columnSources).toArray(ColumnSource[]::new)); + // noinspection unchecked + return (TupleSource) new MultiColumnTupleSource( + Arrays.stream(columnSources).toArray(ColumnSource[]::new)); } private static final Map, Class> TYPE_TO_REINTERPRETED = From 0b3758e2434aaf0cd6994a4c555a94373f5e542d Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 9 Dec 2021 22:49:38 -0500 Subject: [PATCH 05/44] Rename "normal" aggs to "column" aggs. --- .../impl/by/AggregationElementAdapter.java | 16 ++++++++-------- .../table/impl/by/AggregationProcessor.java | 18 +++++++++--------- .../client/impl/BatchTableRequestBuilder.java | 14 +++++++------- .../java/io/deephaven/api/agg/Aggregation.java | 16 ++++++++-------- .../api/agg/AggregationOptimizer.java | 14 +++++++------- .../deephaven/api/agg/AggregationOutputs.java | 8 ++++---- ...Aggregation.java => ColumnAggregation.java} | 9 +++++---- ...gregations.java => ColumnAggregations.java} | 12 ++++++------ .../main/java/io/deephaven/api/agg/Pair.java | 4 ++-- .../io/deephaven/api/agg/spec/AggSpec.java | 6 +++--- .../io/deephaven/api/agg/spec/AggSpecBase.java | 12 ++++++------ 11 files changed, 65 insertions(+), 64 deletions(-) rename table-api/src/main/java/io/deephaven/api/agg/{NormalAggregation.java => ColumnAggregation.java} (59%) rename table-api/src/main/java/io/deephaven/api/agg/{NormalAggregations.java => ColumnAggregations.java} (73%) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java index 3b7f4b2591e..a388c52bad3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java @@ -1,9 +1,9 @@ package io.deephaven.engine.table.impl.by; import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.ColumnAggregation; +import io.deephaven.api.agg.ColumnAggregations; import io.deephaven.api.agg.Count; -import io.deephaven.api.agg.NormalAggregation; -import io.deephaven.api.agg.NormalAggregations; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElementImpl; @@ -31,16 +31,16 @@ public void visit(Count count) { } @Override - public void visit(NormalAggregation normalAgg) { - final AggregationSpec spec = AggregationSpecAdapter.of(normalAgg.spec()); - final MatchPair pair = MatchPair.of(normalAgg.pair()); + public void visit(ColumnAggregation columnAgg) { + final AggregationSpec spec = AggregationSpecAdapter.of(columnAgg.spec()); + final MatchPair pair = MatchPair.of(columnAgg.pair()); out = new AggregationElementImpl(spec, pair); } @Override - public void visit(NormalAggregations normalAggs) { - final AggregationSpec spec = AggregationSpecAdapter.of(normalAggs.spec()); - final MatchPair[] pairs = MatchPair.fromPairs(normalAggs.pairs()); + public void visit(ColumnAggregations columnAggs) { + final AggregationSpec spec = AggregationSpecAdapter.of(columnAggs.spec()); + final MatchPair[] pairs = MatchPair.fromPairs(columnAggs.pairs()); out = new AggregationElementImpl(spec, pairs); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index a92adc05db6..b3ea0f22b6a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -74,7 +74,7 @@ private class Converter implements Aggregation.Visitor, AggSpec.Visitor { private final List> inputSources = new ArrayList<>(); private final List transformers = new ArrayList<>(); - private List resultPairs; + private List resultPairs = List.of(); private int trackedFirstOrLastIndex = -1; private boolean partitionFound = false; // TODO-RWC: Add rollup support @@ -109,17 +109,17 @@ public void visit(@NotNull final Count count) { } @Override - public void visit(@NotNull final NormalAggregation normalAgg) { - resultPairs = List.of(normalAgg.pair()); - normalAgg.spec().walk(this); - resultPairs = null; + public void visit(@NotNull final ColumnAggregation columnAgg) { + resultPairs = List.of(columnAgg.pair()); + columnAgg.spec().walk(this); + resultPairs = List.of(); } @Override - public void visit(@NotNull final NormalAggregations normalAggs) { - resultPairs = normalAggs.pairs(); - normalAggs.spec().walk(this); - resultPairs = null; + public void visit(@NotNull final ColumnAggregations columnAggs) { + resultPairs = columnAggs.pairs(); + columnAggs.spec().walk(this); + resultPairs = List.of(); } // ------------------------------------------------------------------------------------------------------------- diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index dce7ee379ce..8f58088785b 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -12,9 +12,9 @@ import io.deephaven.api.SortColumn.Order; import io.deephaven.api.Strings; import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.ColumnAggregation; +import io.deephaven.api.agg.ColumnAggregations; import io.deephaven.api.agg.Count; -import io.deephaven.api.agg.NormalAggregation; -import io.deephaven.api.agg.NormalAggregations; import io.deephaven.api.agg.Pair; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecAbsSum; @@ -537,14 +537,14 @@ public void visit(Count count) { } @Override - public void visit(NormalAggregation normalAgg) { - out = normalAgg.spec() - .walk(new AggregateAdapter(Collections.singletonList(normalAgg.pair()))).out(); + public void visit(ColumnAggregation columnAgg) { + out = columnAgg.spec() + .walk(new AggregateAdapter(Collections.singletonList(columnAgg.pair()))).out(); } @Override - public void visit(NormalAggregations normalAggs) { - out = normalAggs.spec().walk(new AggregateAdapter(normalAggs.pairs())).out(); + public void visit(ColumnAggregations columnAggs) { + out = columnAggs.spec().walk(new AggregateAdapter(columnAggs.pairs())).out(); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index f30bceeee8f..48c2b0214ff 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -1,6 +1,6 @@ package io.deephaven.api.agg; -import io.deephaven.api.agg.NormalAggregations.Builder; +import io.deephaven.api.agg.ColumnAggregations.Builder; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecFormula; @@ -12,20 +12,20 @@ * * @see io.deephaven.api.TableOperations#aggBy(Collection, Collection) * @see Count - * @see NormalAggregation - * @see NormalAggregations + * @see ColumnAggregation + * @see ColumnAggregations */ public interface Aggregation extends Serializable { - static NormalAggregation of(AggSpec spec, String pair) { - return NormalAggregation.of(spec, Pair.parse(pair)); + static ColumnAggregation of(AggSpec spec, String pair) { + return ColumnAggregation.of(spec, Pair.parse(pair)); } static Aggregation of(AggSpec spec, String... pairs) { if (pairs.length == 1) { return of(spec, pairs[0]); } - final Builder builder = NormalAggregations.builder().spec(spec); + final Builder builder = ColumnAggregations.builder().spec(spec); for (String pair : pairs) { builder.addPairs(Pair.parse(pair)); } @@ -153,8 +153,8 @@ static Aggregation AggWSum(String weightColumn, String... pairs) { interface Visitor { void visit(Count count); - void visit(NormalAggregation normalAgg); + void visit(ColumnAggregation columnAgg); - void visit(NormalAggregations normalAggs); + void visit(ColumnAggregations columnAggs); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java index 5ab505315d7..3390cc13ca4 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java @@ -41,9 +41,9 @@ public List build() { out.add(Count.of((ColumnName) pair)); } } else if (e.getValue().size() == 1) { - out.add(NormalAggregation.of((AggSpec) e.getKey(), e.getValue().get(0))); + out.add(ColumnAggregation.of((AggSpec) e.getKey(), e.getValue().get(0))); } else { - out.add(NormalAggregations.builder().spec((AggSpec) e.getKey()).addAllPairs(e.getValue()).build()); + out.add(ColumnAggregations.builder().spec((AggSpec) e.getKey()).addAllPairs(e.getValue()).build()); } } return out; @@ -55,13 +55,13 @@ public void visit(Count count) { } @Override - public void visit(NormalAggregation normalAgg) { - visitOrder.computeIfAbsent(normalAgg.spec(), k -> new ArrayList<>()).add(normalAgg.pair()); + public void visit(ColumnAggregation columnAgg) { + visitOrder.computeIfAbsent(columnAgg.spec(), k -> new ArrayList<>()).add(columnAgg.pair()); } @Override - public void visit(NormalAggregations normalAggs) { - visitOrder.computeIfAbsent(normalAggs.spec(), k -> new ArrayList<>()) - .addAll(normalAggs.pairs()); + public void visit(ColumnAggregations columnAggs) { + visitOrder.computeIfAbsent(columnAggs.spec(), k -> new ArrayList<>()) + .addAll(columnAggs.pairs()); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java index b145b2b37cc..3311ebbe105 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java @@ -31,12 +31,12 @@ public void visit(Count count) { } @Override - public void visit(NormalAggregation normalAgg) { - out = Stream.of(normalAgg.pair().output()); + public void visit(ColumnAggregation columnAgg) { + out = Stream.of(columnAgg.pair().output()); } @Override - public void visit(NormalAggregations normalAggs) { - out = normalAggs.pairs().stream().map(Pair::output); + public void visit(ColumnAggregations columnAggs) { + out = columnAggs.pairs().stream().map(Pair::output); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/NormalAggregation.java b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java similarity index 59% rename from table-api/src/main/java/io/deephaven/api/agg/NormalAggregation.java rename to table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java index b115e401f73..aa0df65cfc2 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/NormalAggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java @@ -6,14 +6,15 @@ import org.immutables.value.Value.Parameter; /** - * A normal aggregation is an {@link Aggregation} that is composed of a {@link #spec() spec} and a {@link #pair() pair}. + * A ColumnAggregation is an {@link Aggregation} that is composed of a {@link #spec() spec} and a single input/output + * column {@link #pair() pair}. */ @Immutable @SimpleStyle -public abstract class NormalAggregation implements Aggregation { +public abstract class ColumnAggregation implements Aggregation { - public static NormalAggregation of(AggSpec spec, Pair pair) { - return ImmutableNormalAggregation.of(spec, pair); + public static ColumnAggregation of(AggSpec spec, Pair pair) { + return ImmutableColumnAggregation.of(spec, pair); } @Parameter diff --git a/table-api/src/main/java/io/deephaven/api/agg/NormalAggregations.java b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java similarity index 73% rename from table-api/src/main/java/io/deephaven/api/agg/NormalAggregations.java rename to table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java index bf05ddc5b25..43432f7db90 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/NormalAggregations.java +++ b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java @@ -8,15 +8,15 @@ import java.util.List; /** - * Normal aggregations is an {@link Aggregation} that is composed of a {@link #spec() spec} and multiple {@link #pairs() - * pairs}. + * ColumnAggregations is an {@link Aggregation} that is composed of a {@link #spec() spec} and multiple input/output + * column {@link #pairs() pairs}. */ @Immutable @BuildableStyle -public abstract class NormalAggregations implements Aggregation { +public abstract class ColumnAggregations implements Aggregation { public static Builder builder() { - return ImmutableNormalAggregations.builder(); + return ImmutableColumnAggregations.builder(); } public abstract AggSpec spec(); @@ -33,7 +33,7 @@ public final V walk(V visitor) { final void checkSize() { if (pairs().size() < 2) { throw new IllegalArgumentException( - String.format("%s should have at least two pairs", NormalAggregations.class)); + String.format("%s should have at least two pairs", ColumnAggregations.class)); } } @@ -46,6 +46,6 @@ public interface Builder { Builder addAllPairs(Iterable elements); - NormalAggregations build(); + ColumnAggregations build(); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Pair.java b/table-api/src/main/java/io/deephaven/api/agg/Pair.java index bd527bbb50f..9414677fc22 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Pair.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Pair.java @@ -7,8 +7,8 @@ /** * A column pair represents an {@link #input() input} and an {@link #output() output} column. * - * @see NormalAggregation - * @see NormalAggregations + * @see ColumnAggregation + * @see ColumnAggregations */ public interface Pair extends Serializable { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index dd67d1cb4d1..0e8c119d3eb 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -3,14 +3,14 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.NormalAggregation; +import io.deephaven.api.agg.ColumnAggregation; import io.deephaven.api.agg.Pair; import java.util.Arrays; import java.util.Collection; /** - * An aggregation specification represents the configurable options for aggregations. + * An aggregation specification represents the configurable options for column aggregations. */ public interface AggSpec { @@ -138,7 +138,7 @@ static AggSpecWSum wsum(String weightColumn) { return AggSpecWSum.of(ColumnName.of(weightColumn)); } - NormalAggregation aggregation(Pair pair); + ColumnAggregation aggregation(Pair pair); Aggregation aggregation(Pair... pairs); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecBase.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecBase.java index 3facbb66fe9..42a19c93a35 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecBase.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecBase.java @@ -1,8 +1,8 @@ package io.deephaven.api.agg.spec; import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.NormalAggregation; -import io.deephaven.api.agg.NormalAggregations; +import io.deephaven.api.agg.ColumnAggregation; +import io.deephaven.api.agg.ColumnAggregations; import io.deephaven.api.agg.Pair; import java.util.Collection; @@ -10,8 +10,8 @@ public abstract class AggSpecBase implements AggSpec { @Override - public final NormalAggregation aggregation(Pair pair) { - return NormalAggregation.of(this, pair); + public final ColumnAggregation aggregation(Pair pair) { + return ColumnAggregation.of(this, pair); } @Override @@ -19,7 +19,7 @@ public final Aggregation aggregation(Pair... pairs) { if (pairs.length == 1) { return aggregation(pairs[0]); } - return NormalAggregations.builder().spec(this).addPairs(pairs).build(); + return ColumnAggregations.builder().spec(this).addPairs(pairs).build(); } @Override @@ -27,6 +27,6 @@ public final Aggregation aggregation(Collection pairs) { if (pairs.size() == 1) { return aggregation(pairs.iterator().next()); } - return NormalAggregations.builder().spec(this).addAllPairs(pairs).build(); + return ColumnAggregations.builder().spec(this).addAllPairs(pairs).build(); } } From 4ad5c145523ee7adc6fa8cd89eff09389d07f23c Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 9 Dec 2021 23:07:03 -0500 Subject: [PATCH 06/44] Add First/LastRowKey aggs. --- .../impl/select/Formula/FillContext.json | 6 ---- .../table/impl/select/Formula/GetContext.json | 6 ---- .../impl/by/AggregationElementAdapter.java | 19 ++++++++++--- .../table/impl/by/AggregationProcessor.java | 10 +++++++ .../client/impl/BatchTableRequestBuilder.java | 18 ++++++++---- .../io/deephaven/api/agg/Aggregation.java | 13 ++++++++- .../api/agg/AggregationOptimizer.java | 20 +++++++++++++ .../deephaven/api/agg/AggregationOutputs.java | 10 +++++++ .../io/deephaven/api/agg/FirstRowKey.java | 28 +++++++++++++++++++ .../java/io/deephaven/api/agg/LastRowKey.java | 28 +++++++++++++++++++ 10 files changed, 136 insertions(+), 22 deletions(-) delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/FillContext.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/GetContext.json create mode 100644 table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/FillContext.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/FillContext.json deleted file mode 100644 index da98fc5092c..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/FillContext.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.select.Formula$FillContext", - "methods": {}, - "path": "io.deephaven.engine.table.impl.select.Formula.FillContext", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/GetContext.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/GetContext.json deleted file mode 100644 index 0c58fd46608..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/Formula/GetContext.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.select.Formula$GetContext", - "methods": {}, - "path": "io.deephaven.engine.table.impl.select.Formula.GetContext", - "typeName": "interface" -} \ No newline at end of file diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java index a388c52bad3..9706f9a330b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java @@ -1,9 +1,6 @@ package io.deephaven.engine.table.impl.by; -import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.ColumnAggregation; -import io.deephaven.api.agg.ColumnAggregations; -import io.deephaven.api.agg.Count; +import io.deephaven.api.agg.*; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElementImpl; @@ -30,6 +27,20 @@ public void visit(Count count) { out = new AggregationFactory.CountAggregationElement(count.column().name()); } + @Override + public void visit(FirstRowKey firstRowKey) { + out = new AggregationElementImpl( + new KeyOnlyFirstOrLastBySpec(firstRowKey.column().name(), AggType.First), + MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY); + } + + @Override + public void visit(LastRowKey lastRowKey) { + out = new AggregationElementImpl( + new KeyOnlyFirstOrLastBySpec(lastRowKey.column().name(), AggType.Last), + MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY); + } + @Override public void visit(ColumnAggregation columnAgg) { final AggregationSpec spec = AggregationSpecAdapter.of(columnAgg.spec()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index b3ea0f22b6a..6d31b41fb5a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -108,6 +108,16 @@ public void visit(@NotNull final Count count) { inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); } + @Override + public void visit(@NotNull final FirstRowKey firstRowKey) { + visitFirstOrLastAgg(true, firstRowKey.column().name()); + } + + @Override + public void visit(@NotNull final LastRowKey lastRowKey) { + visitFirstOrLastAgg(false, lastRowKey.column().name()); + } + @Override public void visit(@NotNull final ColumnAggregation columnAgg) { resultPairs = List.of(columnAgg.pair()); diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index 8f58088785b..397740e272a 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -11,11 +11,7 @@ import io.deephaven.api.SortColumn; import io.deephaven.api.SortColumn.Order; import io.deephaven.api.Strings; -import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.ColumnAggregation; -import io.deephaven.api.agg.ColumnAggregations; -import io.deephaven.api.agg.Count; -import io.deephaven.api.agg.Pair; +import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecAbsSum; import io.deephaven.api.agg.spec.AggSpecAvg; @@ -536,6 +532,18 @@ public void visit(Count count) { out = Aggregate.newBuilder().setType(AggType.COUNT).setColumnName(count.column().name()).build(); } + @Override + public void visit(FirstRowKey firstRowKey) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + } + + @Override + public void visit(LastRowKey lastRowKey) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + } + @Override public void visit(ColumnAggregation columnAgg) { out = columnAgg.spec() diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 48c2b0214ff..67f30aedc57 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -2,7 +2,6 @@ import io.deephaven.api.agg.ColumnAggregations.Builder; import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.api.agg.spec.AggSpecFormula; import java.io.Serializable; import java.util.Collection; @@ -64,6 +63,10 @@ static Aggregation AggFirst(String... pairs) { return of(AggSpec.first(), pairs); } + static Aggregation AggFirstRowKey(String resultColumn) { + return FirstRowKey.of(resultColumn); + } + static Aggregation AggFormula(String formula, String formulaParam, String... pairs) { return of(AggSpec.formula(formula, formulaParam), pairs); } @@ -76,6 +79,10 @@ static Aggregation AggLast(String... pairs) { return of(AggSpec.last(), pairs); } + static Aggregation AggLastRowKey(String resultColumn) { + return LastRowKey.of(resultColumn); + } + static Aggregation AggMax(String... pairs) { return of(AggSpec.max(), pairs); } @@ -153,6 +160,10 @@ static Aggregation AggWSum(String weightColumn, String... pairs) { interface Visitor { void visit(Count count); + void visit(FirstRowKey firstRowKey); + + void visit(LastRowKey lastRowKey); + void visit(ColumnAggregation columnAgg); void visit(ColumnAggregations columnAggs); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java index 3390cc13ca4..0fa72923d35 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java @@ -15,6 +15,8 @@ public final class AggregationOptimizer implements Aggregation.Visitor { private static final Object COUNT_OBJ = new Object(); + private static final Object FIRST_ROW_KEY_OBJ = new Object(); + private static final Object LAST_ROW_KEY_OBJ = new Object(); /** * Optimizes a collection of {@link Aggregation aggregations} by grouping like-specced aggregations together. The @@ -40,6 +42,14 @@ public List build() { for (Pair pair : e.getValue()) { out.add(Count.of((ColumnName) pair)); } + } else if (e.getKey() == FIRST_ROW_KEY_OBJ) { + for (Pair pair : e.getValue()) { + out.add(FirstRowKey.of((ColumnName) pair)); + } + } else if (e.getKey() == LAST_ROW_KEY_OBJ) { + for (Pair pair : e.getValue()) { + out.add(LastRowKey.of((ColumnName) pair)); + } } else if (e.getValue().size() == 1) { out.add(ColumnAggregation.of((AggSpec) e.getKey(), e.getValue().get(0))); } else { @@ -54,6 +64,16 @@ public void visit(Count count) { visitOrder.computeIfAbsent(COUNT_OBJ, k -> new ArrayList<>()).add(count.column()); } + @Override + public void visit(FirstRowKey firstRowKey) { + visitOrder.computeIfAbsent(FIRST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(firstRowKey.column()); + } + + @Override + public void visit(LastRowKey lastRowKey) { + visitOrder.computeIfAbsent(LAST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(lastRowKey.column()); + } + @Override public void visit(ColumnAggregation columnAgg) { visitOrder.computeIfAbsent(columnAgg.spec(), k -> new ArrayList<>()).add(columnAgg.pair()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java index 3311ebbe105..dbd332bb224 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java @@ -30,6 +30,16 @@ public void visit(Count count) { out = Stream.of(count.column()); } + @Override + public void visit(FirstRowKey firstRowKey) { + out = Stream.of(firstRowKey.column()); + } + + @Override + public void visit(LastRowKey lastRowKey) { + out = Stream.of(lastRowKey.column()); + } + @Override public void visit(ColumnAggregation columnAgg) { out = Stream.of(columnAgg.pair().output()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java b/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java new file mode 100644 index 00000000000..68b6d707bb7 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java @@ -0,0 +1,28 @@ +package io.deephaven.api.agg; + +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.ColumnName; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +@Immutable +@SimpleStyle +public abstract class FirstRowKey implements Aggregation { + + public static FirstRowKey of(ColumnName name) { + return ImmutableFirstRowKey.of(name); + } + + public static FirstRowKey of(String x) { + return of(ColumnName.of(x)); + } + + @Parameter + public abstract ColumnName column(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java b/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java new file mode 100644 index 00000000000..272342dc0a7 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java @@ -0,0 +1,28 @@ +package io.deephaven.api.agg; + +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.ColumnName; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +@Immutable +@SimpleStyle +public abstract class LastRowKey implements Aggregation { + + public static LastRowKey of(ColumnName name) { + return ImmutableLastRowKey.of(name); + } + + public static LastRowKey of(String x) { + return of(ColumnName.of(x)); + } + + @Parameter + public abstract ColumnName column(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } +} From e551c72b50f301bdbb85cc1f0c5fbaffd65afd32 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 10 Dec 2021 00:13:01 -0500 Subject: [PATCH 07/44] Duplication checks in AggregationProcessor --- .../table/impl/by/AggregationProcessor.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 6d31b41fb5a..7d2fb71c0fb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1,5 +1,6 @@ package io.deephaven.engine.table.impl.by; +import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.*; @@ -16,11 +17,9 @@ import io.deephaven.time.DateTime; import org.jetbrains.annotations.NotNull; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; +import java.util.*; import java.util.function.BiFunction; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; @@ -46,6 +45,14 @@ public static AggregationContextFactory of(@NotNull final Collection aggregations) { this.aggregations = aggregations; + final String duplicationErrorMessage = AggregationOutputs.of(aggregations). + collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())). + entrySet().stream().filter(kv -> kv.getValue() > 1). + map(kv -> kv.getKey() + " used " + kv.getValue() + " times"). + collect(Collectors.joining(", ")); + if (!duplicationErrorMessage.isBlank()) { + throw new IllegalArgumentException("Duplicate output columns found: " + duplicationErrorMessage); + } } // ----------------------------------------------------------------------------------------------------------------- From 696bf01f71df468c20a49e0e880b987d3675d62d Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 10 Dec 2021 17:26:07 -0500 Subject: [PATCH 08/44] Cleanups for AggUnique --- .../table/impl/by/AggregationProcessor.java | 2 +- .../table/impl/by/AggregationSpecAdapter.java | 2 +- .../DistinctOperatorFactory.java | 76 +++++++++---------- .../unique/ByteChunkedUniqueOperator.java | 14 ++-- .../unique/ByteRollupUniqueOperator.java | 16 ++-- .../unique/CharChunkedUniqueOperator.java | 14 ++-- .../unique/CharRollupUniqueOperator.java | 16 ++-- .../unique/DoubleChunkedUniqueOperator.java | 14 ++-- .../unique/DoubleRollupUniqueOperator.java | 16 ++-- .../unique/FloatChunkedUniqueOperator.java | 14 ++-- .../unique/FloatRollupUniqueOperator.java | 16 ++-- .../unique/IntChunkedUniqueOperator.java | 14 ++-- .../unique/IntRollupUniqueOperator.java | 16 ++-- .../unique/LongChunkedUniqueOperator.java | 14 ++-- .../unique/LongRollupUniqueOperator.java | 16 ++-- .../unique/ObjectChunkedUniqueOperator.java | 14 ++-- .../unique/ObjectRollupUniqueOperator.java | 16 ++-- .../unique/ShortChunkedUniqueOperator.java | 14 ++-- .../unique/ShortRollupUniqueOperator.java | 16 ++-- .../engine/table/impl/QueryTableTreeTest.java | 6 +- .../barrage/table/BarrageTable.java | 2 +- table-api/build.gradle | 2 + .../io/deephaven/api/agg/Aggregation.java | 8 +- .../io/deephaven/api/agg/spec/AggSpec.java | 8 +- .../deephaven/api/agg/spec/AggSpecUnique.java | 36 +++++---- 25 files changed, 193 insertions(+), 189 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 7d2fb71c0fb..09d5d90ac4f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -237,7 +237,7 @@ public void visit(@NotNull final AggSpecSum sum) { @Override public void visit(@NotNull final AggSpecUnique unique) { visitBasicAgg((t, n) -> getUniqueChunked(t, n, - unique.includeNulls(), unique.onlyNullsValue(), unique.nonUniqueValue(), false, false)); + unique.includeNulls(), null, unique.nonUniqueSentinel(), false, false)); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java index def203b9ddd..76087861af9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java @@ -126,7 +126,7 @@ public void visit(AggSpecSum sum) { @Override public void visit(AggSpecUnique unique) { - out = new UniqueSpec(unique.includeNulls(), unique.onlyNullsValue(), unique.nonUniqueValue()); + out = new UniqueSpec(unique.includeNulls(), null, unique.nonUniqueSentinel()); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java index 79c6794f2af..5d17802f4b2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java @@ -120,61 +120,61 @@ static IterativeChunkedAggregationOperator createDistinct(Class type, String * @return an appropriate operator. */ static IterativeChunkedAggregationOperator createUnique(Class type, String resultName, boolean countNulls, - Object onlyNullsValue, Object nonUniqueValue, + Object onlyNullsSentinel, Object nonUniqueSentinel, boolean exposeInternal, boolean isRollup) { - checkType(resultName, "Only Nulls Value", type, onlyNullsValue); - checkType(resultName, "Non Unique Value", type, nonUniqueValue); + checkType(resultName, "Only Nulls Sentinel", type, onlyNullsSentinel); + checkType(resultName, "Non Unique Sentinel", type, nonUniqueSentinel); if (type == Byte.class || type == byte.class) { - final byte onvAsType = (onlyNullsValue == null) ? NULL_BYTE : ((Number) onlyNullsValue).byteValue(); - final byte nuvAsType = (nonUniqueValue == null) ? NULL_BYTE : ((Number) nonUniqueValue).byteValue(); - return isRollup ? new ByteRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) - : new ByteChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); + final byte onsAsType = (onlyNullsSentinel == null) ? NULL_BYTE : ((Number) onlyNullsSentinel).byteValue(); + final byte nusAsType = (nonUniqueSentinel == null) ? NULL_BYTE : ((Number) nonUniqueSentinel).byteValue(); + return isRollup ? new ByteRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) + : new ByteChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Character.class || type == char.class) { return isRollup ? new CharRollupUniqueOperator(resultName, countNulls, - io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsValue), - io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueValue)) + io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), + io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)) : new CharChunkedUniqueOperator(resultName, countNulls, exposeInternal, - io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsValue), - io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueValue)); + io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), + io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)); } else if (type == Double.class || type == double.class) { - final double onvAsType = (onlyNullsValue == null) ? NULL_DOUBLE : ((Number) onlyNullsValue).doubleValue(); - final double nuvAsType = (nonUniqueValue == null) ? NULL_DOUBLE : ((Number) nonUniqueValue).doubleValue(); - return isRollup ? new DoubleRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) - : new DoubleChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); + final double onsAsType = (onlyNullsSentinel == null) ? NULL_DOUBLE : ((Number) onlyNullsSentinel).doubleValue(); + final double nusAsType = (nonUniqueSentinel == null) ? NULL_DOUBLE : ((Number) nonUniqueSentinel).doubleValue(); + return isRollup ? new DoubleRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) + : new DoubleChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Float.class || type == float.class) { - final float onvAsType = (onlyNullsValue == null) ? NULL_FLOAT : ((Number) onlyNullsValue).floatValue(); - final float nuvAsType = (nonUniqueValue == null) ? NULL_FLOAT : ((Number) nonUniqueValue).floatValue(); - return isRollup ? new FloatRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) - : new FloatChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); + final float onsAsType = (onlyNullsSentinel == null) ? NULL_FLOAT : ((Number) onlyNullsSentinel).floatValue(); + final float nusAsType = (nonUniqueSentinel == null) ? NULL_FLOAT : ((Number) nonUniqueSentinel).floatValue(); + return isRollup ? new FloatRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) + : new FloatChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Integer.class || type == int.class) { - final int onvAsType = (onlyNullsValue == null) ? NULL_INT : ((Number) onlyNullsValue).intValue(); - final int nuvAsType = (nonUniqueValue == null) ? NULL_INT : ((Number) nonUniqueValue).intValue(); - return isRollup ? new IntRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) - : new IntChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); + final int onsAsType = (onlyNullsSentinel == null) ? NULL_INT : ((Number) onlyNullsSentinel).intValue(); + final int nusAsType = (nonUniqueSentinel == null) ? NULL_INT : ((Number) nonUniqueSentinel).intValue(); + return isRollup ? new IntRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) + : new IntChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Long.class || type == long.class || type == DateTime.class) { - final long onvAsType; - final long nuvAsType; + final long onsAsType; + final long nusAsType; if (type == DateTime.class) { - onvAsType = (onlyNullsValue == null) ? NULL_LONG : ((DateTime) onlyNullsValue).getNanos(); - nuvAsType = (nonUniqueValue == null) ? NULL_LONG : ((DateTime) nonUniqueValue).getNanos(); + onsAsType = (onlyNullsSentinel == null) ? NULL_LONG : ((DateTime) onlyNullsSentinel).getNanos(); + nusAsType = (nonUniqueSentinel == null) ? NULL_LONG : ((DateTime) nonUniqueSentinel).getNanos(); } else { - onvAsType = (onlyNullsValue == null) ? NULL_LONG : ((Number) onlyNullsValue).longValue(); - nuvAsType = (nonUniqueValue == null) ? NULL_LONG : ((Number) nonUniqueValue).longValue(); + onsAsType = (onlyNullsSentinel == null) ? NULL_LONG : ((Number) onlyNullsSentinel).longValue(); + nusAsType = (nonUniqueSentinel == null) ? NULL_LONG : ((Number) nonUniqueSentinel).longValue(); } - return isRollup ? new LongRollupUniqueOperator(type, resultName, countNulls, onvAsType, nuvAsType) - : new LongChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onvAsType, nuvAsType); + return isRollup ? new LongRollupUniqueOperator(type, resultName, countNulls, onsAsType, nusAsType) + : new LongChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Short.class || type == short.class) { - final short onvAsType = (onlyNullsValue == null) ? NULL_SHORT : ((Number) onlyNullsValue).shortValue(); - final short nuvAsType = (nonUniqueValue == null) ? NULL_SHORT : ((Number) nonUniqueValue).shortValue(); - return isRollup ? new ShortRollupUniqueOperator(resultName, countNulls, onvAsType, nuvAsType) - : new ShortChunkedUniqueOperator(resultName, countNulls, exposeInternal, onvAsType, nuvAsType); + final short onsAsType = (onlyNullsSentinel == null) ? NULL_SHORT : ((Number) onlyNullsSentinel).shortValue(); + final short nusAsType = (nonUniqueSentinel == null) ? NULL_SHORT : ((Number) nonUniqueSentinel).shortValue(); + return isRollup ? new ShortRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) + : new ShortChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else { - return isRollup ? new ObjectRollupUniqueOperator(type, resultName, countNulls, onlyNullsValue, nonUniqueValue) - : new ObjectChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onlyNullsValue, - nonUniqueValue); + return isRollup ? new ObjectRollupUniqueOperator(type, resultName, countNulls, onlyNullsSentinel, nonUniqueSentinel) + : new ObjectChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onlyNullsSentinel, + nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java index 1f9a81e5831..19681b8437f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java @@ -53,18 +53,18 @@ public class ByteChunkedUniqueOperator implements IterativeChunkedAggregationOpe private final ByteSsmBackedSource ssms; private final ByteArraySource internalResult; private final ColumnSource externalResult; - private final byte noValueKey; - private final byte nonUniqueKey; + private final byte onlyNullsSentinel; + private final byte nonUniqueSentinel; public ByteChunkedUniqueOperator( // region Constructor // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, byte noValueKey, byte nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, byte onlyNullsSentinel, byte nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new ByteSsmBackedSource(); @@ -319,12 +319,12 @@ private void clearSsm(long destination) { private boolean setResult(ByteSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final byte newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java index e939e08e127..f1681facc99 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java @@ -50,8 +50,8 @@ public class ByteRollupUniqueOperator implements IterativeChunkedAggregationOper private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final byte noValueKey; - private final byte nonUniqueKey; + private final byte onlyNullsSentinel; + private final byte nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -61,12 +61,12 @@ public ByteRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - byte noValueKey, - byte nonUniqueKey) { + byte onlyNullsSentinel, + byte nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new ByteSsmBackedSource(); // endregion SsmCreation @@ -546,11 +546,11 @@ private static void flushPrevious(ByteRollupUniqueOperator op) { //region Private Helpers private void updateResult(ByteSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java index 17706dcdf81..13a3d556be1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java @@ -48,18 +48,18 @@ public class CharChunkedUniqueOperator implements IterativeChunkedAggregationOpe private final CharSsmBackedSource ssms; private final CharacterArraySource internalResult; private final ColumnSource externalResult; - private final char noValueKey; - private final char nonUniqueKey; + private final char onlyNullsSentinel; + private final char nonUniqueSentinel; public CharChunkedUniqueOperator( // region Constructor // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, char noValueKey, char nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, char onlyNullsSentinel, char nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new CharSsmBackedSource(); @@ -314,12 +314,12 @@ private void clearSsm(long destination) { private boolean setResult(CharSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final char newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java index 5d1082e5ebf..1899fe7710e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java @@ -45,8 +45,8 @@ public class CharRollupUniqueOperator implements IterativeChunkedAggregationOper private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final char onlyNullsValue; - private final char nonUniqueKey; + private final char onlyNullsSentinel; + private final char nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -56,12 +56,12 @@ public CharRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - char onlyNullsValue, - char nonUniqueKey) { + char onlyNullsSentinel, + char nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.onlyNullsValue = onlyNullsValue; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new CharSsmBackedSource(); // endregion SsmCreation @@ -541,11 +541,11 @@ private static void flushPrevious(CharRollupUniqueOperator op) { //region Private Helpers private void updateResult(CharSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, onlyNullsValue); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java index e09a1c0dd1d..2d37fb196de 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java @@ -53,18 +53,18 @@ public class DoubleChunkedUniqueOperator implements IterativeChunkedAggregationO private final DoubleSsmBackedSource ssms; private final DoubleArraySource internalResult; private final ColumnSource externalResult; - private final double noValueKey; - private final double nonUniqueKey; + private final double onlyNullsSentinel; + private final double nonUniqueSentinel; public DoubleChunkedUniqueOperator( // region Constructor // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, double noValueKey, double nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, double onlyNullsSentinel, double nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new DoubleSsmBackedSource(); @@ -319,12 +319,12 @@ private void clearSsm(long destination) { private boolean setResult(DoubleSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final double newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java index 135d533b433..78965ff35d3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java @@ -50,8 +50,8 @@ public class DoubleRollupUniqueOperator implements IterativeChunkedAggregationOp private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final double noValueKey; - private final double nonUniqueKey; + private final double onlyNullsSentinel; + private final double nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -61,12 +61,12 @@ public DoubleRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - double noValueKey, - double nonUniqueKey) { + double onlyNullsSentinel, + double nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new DoubleSsmBackedSource(); // endregion SsmCreation @@ -546,11 +546,11 @@ private static void flushPrevious(DoubleRollupUniqueOperator op) { //region Private Helpers private void updateResult(DoubleSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java index 0fb579fd9b1..8d026e00a7b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java @@ -53,18 +53,18 @@ public class FloatChunkedUniqueOperator implements IterativeChunkedAggregationOp private final FloatSsmBackedSource ssms; private final FloatArraySource internalResult; private final ColumnSource externalResult; - private final float noValueKey; - private final float nonUniqueKey; + private final float onlyNullsSentinel; + private final float nonUniqueSentinel; public FloatChunkedUniqueOperator( // region Constructor // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, float noValueKey, float nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, float onlyNullsSentinel, float nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new FloatSsmBackedSource(); @@ -319,12 +319,12 @@ private void clearSsm(long destination) { private boolean setResult(FloatSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final float newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java index 0bb052fcd62..72f320812f7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java @@ -50,8 +50,8 @@ public class FloatRollupUniqueOperator implements IterativeChunkedAggregationOpe private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final float noValueKey; - private final float nonUniqueKey; + private final float onlyNullsSentinel; + private final float nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -61,12 +61,12 @@ public FloatRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - float noValueKey, - float nonUniqueKey) { + float onlyNullsSentinel, + float nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new FloatSsmBackedSource(); // endregion SsmCreation @@ -546,11 +546,11 @@ private static void flushPrevious(FloatRollupUniqueOperator op) { //region Private Helpers private void updateResult(FloatSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java index 10c11d02188..8de8237b30a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java @@ -53,18 +53,18 @@ public class IntChunkedUniqueOperator implements IterativeChunkedAggregationOper private final IntSsmBackedSource ssms; private final IntegerArraySource internalResult; private final ColumnSource externalResult; - private final int noValueKey; - private final int nonUniqueKey; + private final int onlyNullsSentinel; + private final int nonUniqueSentinel; public IntChunkedUniqueOperator( // region Constructor // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, int noValueKey, int nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, int onlyNullsSentinel, int nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new IntSsmBackedSource(); @@ -319,12 +319,12 @@ private void clearSsm(long destination) { private boolean setResult(IntSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final int newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java index 001426b5ee2..35ee6f29dd5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java @@ -50,8 +50,8 @@ public class IntRollupUniqueOperator implements IterativeChunkedAggregationOpera private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final int noValueKey; - private final int nonUniqueKey; + private final int onlyNullsSentinel; + private final int nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -61,12 +61,12 @@ public IntRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - int noValueKey, - int nonUniqueKey) { + int onlyNullsSentinel, + int nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new IntSsmBackedSource(); // endregion SsmCreation @@ -546,11 +546,11 @@ private static void flushPrevious(IntRollupUniqueOperator op) { //region Private Helpers private void updateResult(IntSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java index 19b04606c47..b88abee247a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java @@ -57,19 +57,19 @@ public class LongChunkedUniqueOperator implements IterativeChunkedAggregationOpe private final LongSsmBackedSource ssms; private final LongArraySource internalResult; private final ColumnSource externalResult; - private final long noValueKey; - private final long nonUniqueKey; + private final long onlyNullsSentinel; + private final long nonUniqueSentinel; public LongChunkedUniqueOperator( // region Constructor Class type, // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, long noValueKey, long nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, long onlyNullsSentinel, long nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new LongSsmBackedSource(); @@ -328,12 +328,12 @@ private void clearSsm(long destination) { private boolean setResult(LongSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final long newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java index 01f1187e6af..a74bedee32a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java @@ -54,8 +54,8 @@ public class LongRollupUniqueOperator implements IterativeChunkedAggregationOper private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final long noValueKey; - private final long nonUniqueKey; + private final long onlyNullsSentinel; + private final long nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -66,12 +66,12 @@ public LongRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - long noValueKey, - long nonUniqueKey) { + long onlyNullsSentinel, + long nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new LongSsmBackedSource(); // endregion SsmCreation @@ -555,11 +555,11 @@ private static void flushPrevious(LongRollupUniqueOperator op) { //region Private Helpers private void updateResult(LongSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java index 64393f5d392..cb1639300d5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java @@ -53,19 +53,19 @@ public class ObjectChunkedUniqueOperator implements IterativeChunkedAggregationO private final ObjectSsmBackedSource ssms; private final ObjectArraySource internalResult; private final ColumnSource externalResult; - private final Object noValueKey; - private final Object nonUniqueKey; + private final Object onlyNullsSentinel; + private final Object nonUniqueSentinel; public ObjectChunkedUniqueOperator( // region Constructor Class type, // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, Object noValueKey, Object nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, Object onlyNullsSentinel, Object nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new ObjectSsmBackedSource(type); @@ -320,12 +320,12 @@ private void clearSsm(long destination) { private boolean setResult(ObjectSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final Object newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java index fd955f4a8e0..c69c1bf79be 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java @@ -50,8 +50,8 @@ public class ObjectRollupUniqueOperator implements IterativeChunkedAggregationOp private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final Object noValueKey; - private final Object nonUniqueKey; + private final Object onlyNullsSentinel; + private final Object nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -62,12 +62,12 @@ public ObjectRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - Object noValueKey, - Object nonUniqueKey) { + Object onlyNullsSentinel, + Object nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new ObjectSsmBackedSource(type); // endregion SsmCreation @@ -547,11 +547,11 @@ private static void flushPrevious(ObjectRollupUniqueOperator op) { //region Private Helpers private void updateResult(ObjectSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java index 474881c75e1..97cd7a94cb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java @@ -53,18 +53,18 @@ public class ShortChunkedUniqueOperator implements IterativeChunkedAggregationOp private final ShortSsmBackedSource ssms; private final ShortArraySource internalResult; private final ColumnSource externalResult; - private final short noValueKey; - private final short nonUniqueKey; + private final short onlyNullsSentinel; + private final short nonUniqueSentinel; public ShortChunkedUniqueOperator( // region Constructor // endregion Constructor - String name, boolean countNulls, boolean exposeInternal, short noValueKey, short nonUniqueKey) { + String name, boolean countNulls, boolean exposeInternal, short onlyNullsSentinel, short nonUniqueSentinel) { this.name = name; this.countNull = countNulls; this.exposeInternal = exposeInternal; - this.noValueKey = noValueKey; - this.nonUniqueKey = nonUniqueKey; + this.onlyNullsSentinel = onlyNullsSentinel; + this.nonUniqueSentinel = nonUniqueSentinel; // region SsmCreation this.ssms = new ShortSsmBackedSource(); @@ -319,12 +319,12 @@ private void clearSsm(long destination) { private boolean setResult(ShortSegmentedSortedMultiset ssm, long destination) { final boolean resultChanged; if(ssm.isEmpty()) { - resultChanged = internalResult.getAndSetUnsafe(destination, noValueKey) != noValueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, onlyNullsSentinel) != onlyNullsSentinel; } else if(ssm.size() == 1) { final short newValue = ssm.get(0); resultChanged = internalResult.getAndSetUnsafe(destination, newValue) != newValue; } else { - resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueKey) != nonUniqueKey; + resultChanged = internalResult.getAndSetUnsafe(destination, nonUniqueSentinel) != nonUniqueSentinel; } return resultChanged || (exposeInternal && (ssm.getAddedSize() > 0 || ssm.getRemovedSize() > 0)); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java index 17b9c393cde..9ea22ae7fbe 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java @@ -50,8 +50,8 @@ public class ShortRollupUniqueOperator implements IterativeChunkedAggregationOpe private final ColumnSource externalResult; private final Supplier removeContextFactory; private final boolean countNull; - private final short noValueKey; - private final short nonUniqueKey; + private final short onlyNullsSentinel; + private final short nonUniqueSentinel; private UpdateCommitter prevFlusher = null; private WritableRowSet touchedStates; @@ -61,12 +61,12 @@ public ShortRollupUniqueOperator( // endregion Constructor String name, boolean countNulls, - short noValueKey, - short nonUniqueKey) { + short onlyNullsSentinel, + short nonUniqueSentinel) { this.name = name; this.countNull = countNulls; - this.nonUniqueKey = nonUniqueKey; - this.noValueKey = noValueKey; + this.nonUniqueSentinel = nonUniqueSentinel; + this.onlyNullsSentinel = onlyNullsSentinel; // region SsmCreation this.ssms = new ShortSsmBackedSource(); // endregion SsmCreation @@ -546,11 +546,11 @@ private static void flushPrevious(ShortRollupUniqueOperator op) { //region Private Helpers private void updateResult(ShortSegmentedSortedMultiset ssm, long destination) { if(ssm.isEmpty()) { - internalResult.set(destination, noValueKey); + internalResult.set(destination, onlyNullsSentinel); } else if(ssm.size() == 1) { internalResult.set(destination, ssm.get(0)); } else { - internalResult.set(destination, nonUniqueKey); + internalResult.set(destination, nonUniqueSentinel); } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java index def919d6fec..e870492cdec 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java @@ -1296,7 +1296,7 @@ public void testRollupDistinct() { public void testRollupUnique() { testSimpleRollup(List.of(AggUnique("IntCol", "DoubleCol", "FloatNullCol", "StringCol", "BoolCol"))); - testSimpleRollup(List.of(AggUnique(true, "IntCol", "DoubleCol", "FloatNullCol", "StringCol", "BoolCol"))); + testSimpleRollup(List.of(AggUnique(true, null, "IntCol", "DoubleCol", "FloatNullCol", "StringCol", "BoolCol"))); } private void testSimpleRollup(Collection comboAgg) { @@ -1548,7 +1548,7 @@ public void testRollupDistinctIncremental() { public void testRollupUniqueIncremental() { testIncrementalSimple(AggUnique("IntCol")); - testIncrementalSimple(AggUnique(true, "IntCol")); + testIncrementalSimple(AggUnique(true, null, "IntCol")); // TODO (https://github.com/deephaven/deephaven-core/issues/991): Re-enable these sub-tests // testIncrementalSimple(AggUnique(false, -1, -2, "IntCol")); // testIncrementalSimple(AggUnique(true, -1, -2, "IntCol")); @@ -1679,7 +1679,7 @@ private void testRollupIncremental(int seed) { AggDistinct(true, "SDistinctN=StringNulls", "DistinctBoolColN=BoolCol", "DNIntCol=IntSet", "DNLongCol=LongSet", "DNDoubleCol=DoubleSet", "DNFloatCol=FloatSet", "DNCharCol=CharSet", "DNShortCol=ShortSet", "DNByteCol=ByteSet"), - AggUnique(true, "SUniqueN=StringNulls", "UniqueBoolColN=BoolCol", + AggUnique(true, null, "SUniqueN=StringNulls", "UniqueBoolColN=BoolCol", "UNIntCol=IntSet", "UNLongCol=LongSet", "UNDoubleCol=DoubleSet", "UNFloatCol=FloatSet", "UNCharCol=CharSet", "UNShortCol=ShortSet", "UNByteCol=ByteSet")); final EvalNuggetInterface[] en = new EvalNuggetInterface[] { diff --git a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java index 70ad79693bb..45e8a4c6a02 100644 --- a/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java +++ b/extensions/barrage/src/main/java/io/deephaven/extensions/barrage/table/BarrageTable.java @@ -58,7 +58,7 @@ public class BarrageTable extends QueryTable implements BarrageMessage.Listener, /** the capacity that the destSources been set to */ private int capacity = 0; - /** the reinterpretted destination writable sources */ + /** the reinterpreted destination writable sources */ private final WritableColumnSource[] destSources; /** we compact the parent table's key-space and instead redirect; ideal for viewport */ private final WritableRowRedirection rowRedirection; diff --git a/table-api/build.gradle b/table-api/build.gradle index 3dfe34df5ad..1eb117e4a1d 100644 --- a/table-api/build.gradle +++ b/table-api/build.gradle @@ -7,6 +7,8 @@ description = 'The Deephaven table operations API' dependencies { Classpaths.inheritImmutables(project) + compileOnly 'com.google.code.findbugs:jsr305:3.0.2' + Classpaths.inheritJUnitPlatform(project) Classpaths.inheritAssertJ(project) testImplementation 'org.junit.jupiter:junit-jupiter' diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 67f30aedc57..842188e4696 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -135,12 +135,8 @@ static Aggregation AggUnique(String... pairs) { return of(AggSpec.unique(), pairs); } - static Aggregation AggUnique(boolean includeNulls, String... pairs) { - return of(AggSpec.unique(includeNulls), pairs); - } - - static Aggregation AggUnique(boolean includeNulls, Object onlyNullsValue, Object nonUniqueValue, String... pairs) { - return of(AggSpec.unique(includeNulls, onlyNullsValue, nonUniqueValue), pairs); + static Aggregation AggUnique(boolean includeNulls, Object nonUniqueSentinel, String... pairs) { + return of(AggSpec.unique(includeNulls, nonUniqueSentinel), pairs); } static Aggregation AggVar(String... pairs) { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 0e8c119d3eb..d14445a0e12 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -118,12 +118,8 @@ static AggSpecUnique unique() { return AggSpecUnique.of(); } - static AggSpecUnique unique(boolean includeNulls) { - return AggSpecUnique.of(includeNulls); - } - - static AggSpecUnique unique(boolean includeNulls, Object onlyNullsValue, Object nonUniqueValue) { - return AggSpecUnique.of(includeNulls, onlyNullsValue, nonUniqueValue); + static AggSpecUnique unique(boolean includeNulls, Object nonUniqueSentinel) { + return AggSpecUnique.of(includeNulls, nonUniqueSentinel); } static AggSpecVar var() { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java index 8954801fc77..4cf91e8ebbd 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java @@ -4,23 +4,37 @@ import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +import javax.annotation.Nullable; + @Immutable @BuildableStyle public abstract class AggSpecUnique extends AggSpecBase { + /** + * Specify a "unique" aggregation that does not treat {@code null} as a value for purposes of determining if the + * values in a group are unique. If a group is non-empty but contains only {@code null} values, its result will be + * {@code null}. If a group contains more than a single unique value, its result will also be {@code null}. + * + * @return The "unique" aggregation specification + */ public static AggSpecUnique of() { return ImmutableAggSpecUnique.builder().build(); } - public static AggSpecUnique of(boolean includeNulls) { - return ImmutableAggSpecUnique.builder().includeNulls(includeNulls).build(); - } - - public static AggSpecUnique of(boolean includeNulls, Object onlyNullsValue, Object nonUniqueValue) { + /** + * Specify a "unique" aggregation that optionally treats {@code null} as a value for purposes of determining if the + * values in a group are unique. If a group is non-empty but contains only {@code null} values, its result will be + * {@code null}. If a group contains more than a single unique value, its result will be {@code nonUniqueSentinel}. + * + * @param includeNulls Whether {@code null} is treated as a value for determining if the values in a group are + * unique + * @param nonUniqueSentinel Sentinel value to use if a group contains more than a single unique value + * @return The "unique" aggregation specification + */ + public static AggSpecUnique of(boolean includeNulls, Object nonUniqueSentinel) { return ImmutableAggSpecUnique.builder() .includeNulls(includeNulls) - .onlyNullsValue(onlyNullsValue) - .nonUniqueValue(nonUniqueValue) + .nonUniqueSentinel(nonUniqueSentinel) .build(); } @@ -29,13 +43,9 @@ public boolean includeNulls() { return false; } + @Nullable @Default - public Object onlyNullsValue() { - return null; - } - - @Default - public Object nonUniqueValue() { + public Object nonUniqueSentinel() { return null; } From 535236198a9cb8d443c7b7ef3561021279ecab4d Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 10 Dec 2021 18:36:19 -0500 Subject: [PATCH 09/44] Introduce engine-internal rollup-specific Aggregations. --- engine/table/build.gradle | 2 + .../table/impl/by/rollup/NullColumns.java | 43 +++++++++++++++++++ .../table/impl/by/rollup/Partition.java | 24 +++++++++++ .../impl/by/rollup/RollupAggregation.java | 29 +++++++++++++ .../impl/by/rollup/RollupAggregationBase.java | 15 +++++++ 5 files changed, 113 insertions(+) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/Partition.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.java diff --git a/engine/table/build.gradle b/engine/table/build.gradle index dc1b1baf70d..d07f4d964a1 100644 --- a/engine/table/build.gradle +++ b/engine/table/build.gradle @@ -44,6 +44,8 @@ dependencies { Classpaths.inheritJavaParser(project) Classpaths.inheritAutoService(project) + Classpaths.inheritImmutables(project) + compileOnly 'com.google.code.findbugs:jsr305:3.0.2' runtimeOnly project(':engine-tuplesource') diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java new file mode 100644 index 00000000000..88577564e4d --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java @@ -0,0 +1,43 @@ +package io.deephaven.engine.table.impl.by.rollup; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value; + +import java.util.Map; + +/** + * {@link RollupAggregation} that allows columns to be nulled-out at higher aggregation levels. + */ +@Value.Immutable +@BuildableStyle +public abstract class NullColumns extends RollupAggregationBase { + + public static Builder builder() { + return ImmutableNullColumns.builder(); + } + + public abstract Map> resultColumns(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } + + @Value.Check + final void checkNonEmpty() { + if (resultColumns().isEmpty()) { + throw new IllegalArgumentException( + String.format("%s should have at least one result column", NullColumns.class)); + } + } + + public interface Builder { + + Builder putResultColumns(String name, Class type); + + Builder putResultColumns(Map> resultColumns); + + NullColumns build(); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/Partition.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/Partition.java new file mode 100644 index 00000000000..858aa7b5f2c --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/Partition.java @@ -0,0 +1,24 @@ +package io.deephaven.engine.table.impl.by.rollup; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value; + +/** + * {@link RollupAggregation} that represents a partitioning of the input table. + */ +@Value.Immutable +@BuildableStyle +public abstract class Partition extends RollupAggregationBase { + + public static Partition of(boolean includeConstituents) { + return ImmutablePartition.builder().includeConstituents(includeConstituents).build(); + } + + public abstract boolean includeConstituents(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java new file mode 100644 index 00000000000..ee1c8efadf4 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java @@ -0,0 +1,29 @@ +package io.deephaven.engine.table.impl.by.rollup; + +import io.deephaven.api.agg.Aggregation; + +import java.util.Map; + +/** + * Rollup-specific {@link Aggregation aggregations}. + */ +public interface RollupAggregation extends Aggregation { + + static RollupAggregation nullColumns(Map> resultColumns) { + return NullColumns.builder().putResultColumns(resultColumns).build(); + } + + static RollupAggregation partition(boolean includeConstituents) { + return Partition.of(includeConstituents); + } + + V walk(V visitor); + + V walk(V visitor); + + interface Visitor extends Aggregation.Visitor { + void visit(NullColumns nullColumns); + + void visit(Partition partition); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.java new file mode 100644 index 00000000000..667c21ae831 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.java @@ -0,0 +1,15 @@ +package io.deephaven.engine.table.impl.by.rollup; + +import io.deephaven.api.agg.Aggregation; + +public abstract class RollupAggregationBase implements RollupAggregation { + + @Override + public final V walk(V visitor) { + if (visitor instanceof Visitor) { + walk((Visitor) visitor); + return visitor; + } + throw new UnsupportedOperationException("Cannot walk a RollupAggregation without a RollupAggregation.Visitor"); + } +} From 5556adbbe7d2241e8e87ff36758e37ade9f31709 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 14 Jan 2022 10:39:03 -0500 Subject: [PATCH 10/44] Delete some unused imports in RowSetFactory --- .../main/java/io/deephaven/engine/rowset/RowSetFactory.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetFactory.java b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetFactory.java index 2d623e0a1ec..1b6e34c70d1 100644 --- a/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetFactory.java +++ b/engine/rowset/src/main/java/io/deephaven/engine/rowset/RowSetFactory.java @@ -1,10 +1,6 @@ package io.deephaven.engine.rowset; import gnu.trove.list.TLongList; -import io.deephaven.engine.rowset.RowSet; -import io.deephaven.engine.rowset.RowSetBuilderRandom; -import io.deephaven.engine.rowset.RowSetBuilderSequential; -import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.impl.AdaptiveRowSetBuilderRandom; import io.deephaven.engine.rowset.impl.BasicRowSetBuilderSequential; import io.deephaven.engine.rowset.impl.WritableRowSetImpl; From f652d3a4662ce24052d38f276ca0b170dbabfa71 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 14 Jan 2022 17:18:32 -0500 Subject: [PATCH 11/44] Broken WIP on rollup support in AggregationProcessor --- .../table/impl/by/AggregationProcessor.java | 218 ++++++++++++++++-- 1 file changed, 197 insertions(+), 21 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 09d5d90ac4f..7b8da92d698 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -12,6 +12,9 @@ import io.deephaven.engine.table.impl.BaseTable; import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.TupleSourceFactory; +import io.deephaven.engine.table.impl.by.rollup.NullColumns; +import io.deephaven.engine.table.impl.by.rollup.Partition; +import io.deephaven.engine.table.impl.by.rollup.RollupAggregation; import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.time.DateTime; @@ -37,17 +40,28 @@ public class AggregationProcessor implements AggregationContextFactory { * @param aggregations The {@link Aggregation aggregations} * @return The {@link AggregationContextFactory} */ - public static AggregationContextFactory of(@NotNull final Collection aggregations) { - return new AggregationProcessor(aggregations); + public static AggregationContextFactory of(@NotNull final Collection aggregations, + @NotNull final Type type) { + return new AggregationProcessor(aggregations, type); + } + + public enum Type { + STANDARD, + ROLLUP_BASE, + ROLLUP_REAGGREGATED } private final Collection aggregations; + private final Type type; - private AggregationProcessor(@NotNull final Collection aggregations) { + private AggregationProcessor( + @NotNull final Collection aggregations, + @NotNull final Type type) { this.aggregations = aggregations; + this.type = type; final String duplicationErrorMessage = AggregationOutputs.of(aggregations). - collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())). - entrySet().stream().filter(kv -> kv.getValue() > 1). + collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream(). + filter(kv -> kv.getValue() > 1). map(kv -> kv.getKey() + " used " + kv.getValue() + " times"). collect(Collectors.joining(", ")); if (!duplicationErrorMessage.isBlank()) { @@ -61,31 +75,37 @@ private AggregationProcessor(@NotNull final Collection ag @Override public AggregationContext makeAggregationContext(@NotNull Table table, @NotNull String... groupByColumnNames) { - return new Converter(table, groupByColumnNames).build(); + switch (type) { + case STANDARD: + return new StandardConverter(table, groupByColumnNames).build(); + case ROLLUP_BASE: + case ROLLUP_REAGGREGATED: + default: + throw new UnsupportedOperationException("Unsupported type " + type); + } } /** * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an - * {@link AggregationContext}. Accumulates state by visiting each aggregation. + * {@link AggregationContext} for standard aggregations. Accumulates state by visiting each aggregation. */ - private class Converter implements Aggregation.Visitor, AggSpec.Visitor { + private class StandardConverter implements Aggregation.Visitor, AggSpec.Visitor { - private final Table table; - private final String[] groupByColumnNames; + protected final Table table; + protected final String[] groupByColumnNames; - private final boolean isAddOnly; - private final boolean isStream; + protected final boolean isAddOnly; + protected final boolean isStream; - private final List operators = new ArrayList<>(); - private final List inputColumnNames = new ArrayList<>(); - private final List> inputSources = new ArrayList<>(); - private final List transformers = new ArrayList<>(); + protected final List operators = new ArrayList<>(); + protected final List inputColumnNames = new ArrayList<>(); + protected final List> inputSources = new ArrayList<>(); + protected final List transformers = new ArrayList<>(); - private List resultPairs = List.of(); - private int trackedFirstOrLastIndex = -1; - private boolean partitionFound = false; // TODO-RWC: Add rollup support + protected List resultPairs = List.of(); + protected int trackedFirstOrLastIndex = -1; - private Converter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + private StandardConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { this.table = table; this.groupByColumnNames = groupByColumnNames; isAddOnly = ((BaseTable) table).isAddOnly(); @@ -301,7 +321,8 @@ private void visitMinOrMaxAgg(final boolean isMin) { private void visitFirstOrLastAgg(final boolean isFirst, final String exposeRedirectionAs) { if (exposeRedirectionAs != null) { - streamUnsupported((isFirst ? "First" : "Last") + " with exposed row redirections (e.g. for rollup())"); + streamUnsupported((isFirst ? "First" : "Last") + + " with exposed row redirections (e.g. for rollup(), AggFirstRowKey, or AggLastRowKey)"); } final MatchPair[] resultMatchPairs = MatchPair.fromPairs(resultPairs); if (table.isRefreshing()) { @@ -369,6 +390,161 @@ private void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn so } } + /** + * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an + * {@link AggregationContext} for rollup base aggregations. + */ + private class RollupBaseConverter extends StandardConverter implements RollupAggregation.Visitor { + + protected RollupBaseConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + super(table, groupByColumnNames); + } + + // ------------------------------------------------------------------------------------------------------------- + // RollupAggregation.Visitor + // ------------------------------------------------------------------------------------------------------------- + + @Override + public void visit(@NotNull final NullColumns nullColumns) { + // TODO-RWC: This is not for base! + transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns()); + } + + @Override + public void visit(@NotNull final Partition partition) { + + } + + @Override + public void visit(@NotNull final Count count) { + + } + + @Override + public void visit(@NotNull final FirstRowKey firstRowKey) { + + } + + @Override + public void visit(@NotNull final LastRowKey lastRowKey) { + + } + + @Override + public void visit(@NotNull final ColumnAggregation columnAgg) { + + } + + @Override + public void visit(@NotNull final ColumnAggregations columnAggs) { + + } + + // ------------------------------------------------------------------------------------------------------------- + // AggSpec.Visitor + // ------------------------------------------------------------------------------------------------------------- + + @Override + public void visit(@NotNull final AggSpecAbsSum absSum) { + + } + + @Override + public void visit(@NotNull final AggSpecCountDistinct countDistinct) { + + } + + @Override + public void visit(@NotNull final AggSpecDistinct distinct) { + + } + + @Override + public void visit(@NotNull final AggSpecGroup group) { + + } + + @Override + public void visit(@NotNull final AggSpecAvg avg) { + + } + + @Override + public void visit(@NotNull final AggSpecFirst first) { + + } + + @Override + public void visit(@NotNull final AggSpecFormula formula) { + + } + + @Override + public void visit(@NotNull final AggSpecLast last) { + + } + + @Override + public void visit(@NotNull final AggSpecMax max) { + + } + + @Override + public void visit(@NotNull final AggSpecMedian median) { + + } + + @Override + public void visit(@NotNull final AggSpecMin min) { + + } + + @Override + public void visit(@NotNull final AggSpecPercentile pct) { + + } + + @Override + public void visit(@NotNull final AggSpecSortedFirst sortedFirst) { + + } + + @Override + public void visit(@NotNull final AggSpecSortedLast sortedLast) { + + } + + @Override + public void visit(@NotNull final AggSpecStd std) { + + } + + @Override + public void visit(@NotNull final AggSpecSum sum) { + + } + + @Override + public void visit(@NotNull final AggSpecUnique unique) { + + } + + @Override + public void visit(@NotNull final AggSpecWAvg wAvg) { + + } + + @Override + public void visit(@NotNull final AggSpecWSum wSum) { + + } + + @Override + public void visit(@NotNull final AggSpecVar var) { + + } + } + private static ColumnSource maybeReinterpretDateTimeAsLong(@NotNull final ColumnSource inputSource) { return inputSource.getType() == DateTime.class ? ReinterpretUtils.dateTimeToLongSource(inputSource) From 52bec991515e056ed10e20c51e3af130ef3416a2 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 24 Jan 2022 22:51:33 -0500 Subject: [PATCH 12/44] More WIP on Agg Processor for Rollups. --- .../engine/table/impl/QueryTable.java | 2 +- .../table/impl/by/AggregationFactory.java | 29 +- .../table/impl/by/AggregationProcessor.java | 658 +++++++++++++----- .../engine/table/impl/by/BaseConverter.java | 19 + .../impl/by/BigDecimalChunkedAvgOperator.java | 2 +- .../impl/by/BigDecimalChunkedVarOperator.java | 2 +- .../impl/by/BigIntegerChunkedAvgOperator.java | 7 +- .../impl/by/BigIntegerChunkedVarOperator.java | 2 +- .../table/impl/by/ByteChunkedAvgOperator.java | 2 +- .../table/impl/by/ByteChunkedVarOperator.java | 2 +- .../table/impl/by/CharChunkedAvgOperator.java | 2 +- .../table/impl/by/CharChunkedVarOperator.java | 2 +- .../impl/by/DoubleChunkedAvgOperator.java | 2 +- .../impl/by/DoubleChunkedVarOperator.java | 2 +- .../engine/table/impl/by/FirstBySpecImpl.java | 2 +- .../impl/by/FloatChunkedAvgOperator.java | 2 +- .../impl/by/FloatChunkedVarOperator.java | 2 +- .../impl/by/FpChunkedNonNormalCounter.java | 2 +- .../table/impl/by/IntChunkedAvgOperator.java | 2 +- .../table/impl/by/IntChunkedVarOperator.java | 2 +- .../table/impl/by/IterativeIndexSpec.java | 4 +- .../engine/table/impl/by/LastBySpecImpl.java | 2 +- .../table/impl/by/LongChunkedAvgOperator.java | 2 +- .../table/impl/by/LongChunkedVarOperator.java | 2 +- .../impl/by/ShortChunkedAvgOperator.java | 2 +- .../impl/by/ShortChunkedVarOperator.java | 2 +- .../impl/by/TrackingFirstBySpecImpl.java | 2 +- .../table/impl/by/TrackingLastBySpecImpl.java | 2 +- .../ByteChunkedCountDistinctOperator.java | 4 +- .../ByteRollupCountDistinctOperator.java | 4 +- .../CharChunkedCountDistinctOperator.java | 4 +- .../CharRollupCountDistinctOperator.java | 4 +- .../DoubleChunkedCountDistinctOperator.java | 4 +- .../DoubleRollupCountDistinctOperator.java | 4 +- .../FloatChunkedCountDistinctOperator.java | 4 +- .../FloatRollupCountDistinctOperator.java | 4 +- .../IntChunkedCountDistinctOperator.java | 4 +- .../count/IntRollupCountDistinctOperator.java | 4 +- .../LongChunkedCountDistinctOperator.java | 4 +- .../LongRollupCountDistinctOperator.java | 4 +- .../ObjectChunkedCountDistinctOperator.java | 4 +- .../ObjectRollupCountDistinctOperator.java | 4 +- .../ShortChunkedCountDistinctOperator.java | 4 +- .../ShortRollupCountDistinctOperator.java | 4 +- .../distinct/ByteChunkedDistinctOperator.java | 4 +- .../distinct/ByteRollupDistinctOperator.java | 4 +- .../distinct/CharChunkedDistinctOperator.java | 4 +- .../distinct/CharRollupDistinctOperator.java | 4 +- .../DoubleChunkedDistinctOperator.java | 4 +- .../DoubleRollupDistinctOperator.java | 4 +- .../FloatChunkedDistinctOperator.java | 4 +- .../distinct/FloatRollupDistinctOperator.java | 4 +- .../distinct/IntChunkedDistinctOperator.java | 4 +- .../distinct/IntRollupDistinctOperator.java | 4 +- .../distinct/LongChunkedDistinctOperator.java | 5 +- .../distinct/LongRollupDistinctOperator.java | 5 +- .../ObjectChunkedDistinctOperator.java | 4 +- .../ObjectRollupDistinctOperator.java | 4 +- .../ShortChunkedDistinctOperator.java | 4 +- .../distinct/ShortRollupDistinctOperator.java | 4 +- .../unique/ByteChunkedUniqueOperator.java | 4 +- .../unique/ByteRollupUniqueOperator.java | 4 +- .../unique/CharChunkedUniqueOperator.java | 4 +- .../unique/CharRollupUniqueOperator.java | 4 +- .../unique/DoubleChunkedUniqueOperator.java | 4 +- .../unique/DoubleRollupUniqueOperator.java | 4 +- .../unique/FloatChunkedUniqueOperator.java | 4 +- .../unique/FloatRollupUniqueOperator.java | 4 +- .../unique/IntChunkedUniqueOperator.java | 4 +- .../unique/IntRollupUniqueOperator.java | 4 +- .../unique/LongChunkedUniqueOperator.java | 5 +- .../unique/LongRollupUniqueOperator.java | 5 +- .../unique/ObjectChunkedUniqueOperator.java | 4 +- .../unique/ObjectRollupUniqueOperator.java | 4 +- .../unique/ShortChunkedUniqueOperator.java | 4 +- .../unique/ShortRollupUniqueOperator.java | 4 +- 76 files changed, 631 insertions(+), 328 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index 6aba0a9f769..1cc8775056b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -506,7 +506,7 @@ public Table rollup(AggregationFactory aggregationFactory, boolean includeConsti } final String[] rollupsToDrop = lastLevel.getColumnSourceMap().keySet().stream() - .filter(cn -> cn.endsWith(AggregationFactory.ROLLUP_COLUMN_SUFFIX)).toArray(String[]::new); + .filter(cn -> cn.endsWith(RollupConstants.ROLLUP_COLUMN_SUFFIX)).toArray(String[]::new); final QueryTable finalTable = (QueryTable) lastLevel.dropColumns(rollupsToDrop); final Object reverseLookup = Require.neqNull(lastLevel.getAttribute(REVERSE_LOOKUP_ATTRIBUTE), "REVERSE_LOOKUP_ATTRIBUTE"); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java index a5aac6ae073..9dcc35f92f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java @@ -34,20 +34,11 @@ * {@link AggregationContextFactory} for use internally by the implementation of {@link Table#aggBy}. */ public class AggregationFactory implements AggregationSpec { - static final String ROLLUP_RUNNING_SUM_COLUMN_ID = "_RS_"; - static final String ROLLUP_RUNNING_SUM2_COLUMN_ID = "_RS2_"; - static final String ROLLUP_NONNULL_COUNT_COLUMN_ID = "_NNC_"; - static final String ROLLUP_NAN_COLUMN_ID = "_NaN_"; - static final String ROLLUP_PIC_COLUMN_ID = "_PIC_"; - static final String ROLLUP_NIC_COLUMN_ID = "_NIC_"; - public static final String ROLLUP_DISTINCT_SSM_COLUMN_ID = "_SSM_"; private final List underlyingAggregations = new ArrayList<>(); private final boolean isRollup; private final boolean secondLevel; - public static final String ROLLUP_COLUMN_SUFFIX = "__ROLLUP__"; - /** * Create a factory for performing rollups. */ @@ -532,7 +523,7 @@ public AggregationContextFactory makeAggregationContextFactory() { if (sortedFirstOrLastByFactory.secondRollup && sortedFirstOrLastByFactory.getSortColumnNames().length == 1 && sortedFirstOrLastByFactory.getSortColumnNames()[0] - .endsWith(ROLLUP_COLUMN_SUFFIX)) { + .endsWith(RollupConstants.ROLLUP_COLUMN_SUFFIX)) { updatedMatchPairs = Arrays.copyOf(comboMatchPairs, comboMatchPairs.length + 1); final String redirectionName = sortedFirstOrLastByFactory.getSortColumnNames()[0]; updatedMatchPairs[updatedMatchPairs.length - 1] = @@ -563,17 +554,17 @@ public AggregationContextFactory makeAggregationContextFactory() { final boolean isStdVar = isStd || isVar; if (isAverage || isStdVar) { final String runningSumName = - mp.leftColumn() + ROLLUP_RUNNING_SUM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_RUNNING_SUM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String runningSum2Name = - mp.leftColumn() + ROLLUP_RUNNING_SUM2_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_RUNNING_SUM2_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String nonNullName = - mp.leftColumn() + ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_NONNULL_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String nanName = - mp.leftColumn() + ROLLUP_NAN_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_NAN_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String picName = - mp.leftColumn() + ROLLUP_PIC_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_PIC_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String nicName = - mp.leftColumn() + ROLLUP_NIC_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_NIC_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final boolean isFloatingPoint = table.hasColumns(nanName); @@ -740,7 +731,7 @@ public AggregationContextFactory makeAggregationContextFactory() { return; } else if (isCountDistinct || isDistinct || isAggUnique) { final String ssmColName = - mp.leftColumn() + ROLLUP_DISTINCT_SSM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final ColumnSource> ssmSource = table.getColumnSource(ssmColName); final ColumnSource lastLevelResult = table.getColumnSource(mp.leftColumn()); @@ -925,7 +916,7 @@ public AggregationContextFactory makeAggregationContextFactory() { final QueryTable adjustedTable; final List columnsToDrop = parentTable.getDefinition().getColumnStream().map(ColumnDefinition::getName) - .filter(cn -> cn.endsWith(ROLLUP_COLUMN_SUFFIX)).collect(Collectors.toList()); + .filter(cn -> cn.endsWith(RollupConstants.ROLLUP_COLUMN_SUFFIX)).collect(Collectors.toList()); if (!columnsToDrop.isEmpty()) { adjustedTable = (QueryTable) parentTable.dropColumns(columnsToDrop); } else { @@ -993,7 +984,7 @@ private static UnsupportedOperationException streamUnsupported(@NotNull final St @NotNull private static String makeRedirectionName(IterativeIndexSpec inputAggregationStateFactory) { return IterativeIndexSpec.ROW_REDIRECTION_PREFIX + inputAggregationStateFactory.rollupColumnIdentifier - + ROLLUP_COLUMN_SUFFIX; + + RollupConstants.ROLLUP_COLUMN_SUFFIX; } private static class RollupTableMapAndReverseLookupAttributeSetter implements AggregationContextTransformer { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 7b8da92d698..6cedb382164 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -5,28 +5,30 @@ import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.*; import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.BaseTable; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.TupleSourceFactory; +import io.deephaven.datastructures.util.SmartKey; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.*; import io.deephaven.engine.table.impl.by.rollup.NullColumns; import io.deephaven.engine.table.impl.by.rollup.Partition; import io.deephaven.engine.table.impl.by.rollup.RollupAggregation; import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; +import io.deephaven.engine.table.impl.sources.SingleValueObjectColumnSource; import io.deephaven.time.DateTime; +import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; import java.util.*; import java.util.function.BiFunction; +import java.util.function.ToIntFunction; import java.util.stream.Collectors; import java.util.stream.IntStream; import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; +import static io.deephaven.engine.table.Table.REVERSE_LOOKUP_ATTRIBUTE; +import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; import static io.deephaven.engine.table.impl.by.IterativeOperatorSpec.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_COLUMN_SUFFIX; /** * Conversion tool to generate an {@link AggregationContextFactory} for a collection of {@link Aggregation @@ -41,14 +43,12 @@ public class AggregationProcessor implements AggregationContextFactory { * @return The {@link AggregationContextFactory} */ public static AggregationContextFactory of(@NotNull final Collection aggregations, - @NotNull final Type type) { + @NotNull final Type type) { return new AggregationProcessor(aggregations, type); } public enum Type { - STANDARD, - ROLLUP_BASE, - ROLLUP_REAGGREGATED + STANDARD, ROLLUP_BASE, ROLLUP_REAGGREGATED } private final Collection aggregations; @@ -59,11 +59,10 @@ private AggregationProcessor( @NotNull final Type type) { this.aggregations = aggregations; this.type = type; - final String duplicationErrorMessage = AggregationOutputs.of(aggregations). - collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream(). - filter(kv -> kv.getValue() > 1). - map(kv -> kv.getKey() + " used " + kv.getValue() + " times"). - collect(Collectors.joining(", ")); + final String duplicationErrorMessage = AggregationOutputs.of(aggregations) + .collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream() + .filter(kv -> kv.getValue() > 1).map(kv -> kv.getKey() + " used " + kv.getValue() + " times") + .collect(Collectors.joining(", ")); if (!duplicationErrorMessage.isBlank()) { throw new IllegalArgumentException("Duplicate output columns found: " + duplicationErrorMessage); } @@ -79,40 +78,46 @@ public AggregationContext makeAggregationContext(@NotNull Table table, @NotNull case STANDARD: return new StandardConverter(table, groupByColumnNames).build(); case ROLLUP_BASE: + return new RollupBaseConverter(table, groupByColumnNames).build(); case ROLLUP_REAGGREGATED: + return new RollupReaggregatedConverter(table, groupByColumnNames).build(); default: throw new UnsupportedOperationException("Unsupported type " + type); } } + // ----------------------------------------------------------------------------------------------------------------- + // Converter Framework + // ----------------------------------------------------------------------------------------------------------------- + /** - * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an - * {@link AggregationContext} for standard aggregations. Accumulates state by visiting each aggregation. + * Base class for conversion from a collection of {@link Aggregation aggregations} to an {@link AggregationContext} + * for {@code aggregations}. Accumulates state by visiting each aggregation. */ - private class StandardConverter implements Aggregation.Visitor, AggSpec.Visitor { + private abstract class Converter implements Aggregation.Visitor, AggSpec.Visitor { - protected final Table table; - protected final String[] groupByColumnNames; + final QueryTable table; + final String[] groupByColumnNames; - protected final boolean isAddOnly; - protected final boolean isStream; + final boolean isAddOnly; + final boolean isStream; - protected final List operators = new ArrayList<>(); - protected final List inputColumnNames = new ArrayList<>(); - protected final List> inputSources = new ArrayList<>(); - protected final List transformers = new ArrayList<>(); + final List operators = new ArrayList<>(); + final List inputColumnNames = new ArrayList<>(); + final List> inputSources = new ArrayList<>(); + final List transformers = new ArrayList<>(); - protected List resultPairs = List.of(); - protected int trackedFirstOrLastIndex = -1; + List resultPairs = List.of(); + int trackedFirstOrLastIndex = -1; - private StandardConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { - this.table = table; + private Converter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + this.table = (QueryTable) table.coalesce(); this.groupByColumnNames = groupByColumnNames; - isAddOnly = ((BaseTable) table).isAddOnly(); - isStream = ((BaseTable) table).isStream(); + isAddOnly = this.table.isAddOnly(); + isStream = this.table.isStream(); } - private AggregationContext build() { + AggregationContext build() { for (final Aggregation aggregation : aggregations) { aggregation.walk(this); } @@ -124,41 +129,181 @@ private AggregationContext build() { transformers.toArray(AggregationContextTransformer[]::new)); } - // ------------------------------------------------------------------------------------------------------------- - // Aggregation.Visitor - // ------------------------------------------------------------------------------------------------------------- - - @Override - public void visit(@NotNull final Count count) { - operators.add(new CountAggregationOperator(count.column().name())); - inputSources.add(null); + final void addNoInputOperator(@NotNull final IterativeChunkedAggregationOperator operator) { + operators.add(operator); inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); + inputSources.add(null); } - @Override - public void visit(@NotNull final FirstRowKey firstRowKey) { - visitFirstOrLastAgg(true, firstRowKey.column().name()); + void streamUnsupported(@NotNull final String operationName) { + if (!isStream) { + return; + } + throw new UnsupportedOperationException(String.format( + "Stream tables do not support Agg%s; use StreamTableTools.streamToAppendOnlyTable to accumulate full history", + operationName)); } + // ------------------------------------------------------------------------------------------------------------- + // Partial Aggregation.Visitor (for cases common to all types) + // ------------------------------------------------------------------------------------------------------------- + @Override - public void visit(@NotNull final LastRowKey lastRowKey) { - visitFirstOrLastAgg(false, lastRowKey.column().name()); + public final void visit(@NotNull final Count count) { + addNoInputOperator(new CountAggregationOperator(count.column().name())); } + // ------------------------------------------------------------------------------------------------------------- + // Partial AggSpec.Visitor (for cases common to all types) + // ------------------------------------------------------------------------------------------------------------- + @Override - public void visit(@NotNull final ColumnAggregation columnAgg) { + public final void visit(@NotNull final ColumnAggregation columnAgg) { resultPairs = List.of(columnAgg.pair()); columnAgg.spec().walk(this); resultPairs = List.of(); } @Override - public void visit(@NotNull final ColumnAggregations columnAggs) { + public final void visit(@NotNull final ColumnAggregations columnAggs) { resultPairs = columnAggs.pairs(); columnAggs.spec().walk(this); resultPairs = List.of(); } + // ------------------------------------------------------------------------------------------------------------- + // Helpers for visitors + // ------------------------------------------------------------------------------------------------------------- + + void visitBasicAgg(BiFunction, String, IterativeChunkedAggregationOperator> operatorFactory) { + for (final Pair pair : resultPairs) { + final String inputName = pair.input().name(); + final String resultName = pair.output().name(); + final ColumnSource rawInputSource = table.getColumnSource(inputName); + final Class type = rawInputSource.getType(); + final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + + operators.add(operatorFactory.apply(type, resultName)); + inputColumnNames.add(new String[] {inputName}); + inputSources.add(inputSource); + } + } + + void visitMinOrMaxAgg(final boolean isMin) { + for (final Pair pair : resultPairs) { + final String inputName = pair.input().name(); + final String resultName = pair.output().name(); + final ColumnSource rawInputSource = table.getColumnSource(inputName); + final Class type = rawInputSource.getType(); + final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + + IntStream.range(0, inputSources.size()) + .filter(index -> (inputSources.get(index) == inputSource) + && (operators.get(index) instanceof SsmChunkedMinMaxOperator)) + .findFirst().ifPresentOrElse( + (final int priorMinMaxIndex) -> { + final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = + (SsmChunkedMinMaxOperator) operators.get(priorMinMaxIndex); + operators.add(ssmChunkedMinMaxOperator.makeSecondaryOperator(isMin, resultName)); + inputColumnNames.add(new String[] {inputName}); + inputSources.add(null); + }, + () -> { + operators.add(getMinMaxChunked(type, resultName, isMin, isAddOnly || isStream)); + inputColumnNames.add(new String[] {inputName}); + inputSources.add(inputSource); + }); + } + } + + void visitFirstOrLastAgg(final boolean isFirst, final String exposeRedirectionAs) { + if (exposeRedirectionAs != null) { + streamUnsupported((isFirst ? "First" : "Last") + + " with exposed row redirections (e.g. for rollup(), AggFirstRowKey, or AggLastRowKey)"); + } + final MatchPair[] resultMatchPairs = MatchPair.fromPairs(resultPairs); + final IterativeChunkedAggregationOperator operator; + if (table.isRefreshing()) { + if (isAddOnly) { + operator = new AddOnlyFirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, + exposeRedirectionAs); + } else if (isStream) { + operator = isFirst + ? new StreamFirstChunkedOperator(resultMatchPairs, table) + : new StreamLastChunkedOperator(resultMatchPairs, table); + } else { + if (trackedFirstOrLastIndex >= 0) { + operator = ((FirstOrLastChunkedOperator) operators.get(trackedFirstOrLastIndex)) + .makeSecondaryOperator(isFirst, resultMatchPairs, table, exposeRedirectionAs); + } else { + trackedFirstOrLastIndex = operators.size(); + operator = new FirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, + exposeRedirectionAs); + } + } + } else { + operator = new StaticFirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, exposeRedirectionAs); + } + addNoInputOperator(operator); + } + + void visitSortedFirstOrLastAgg(@NotNull final List sortColumns, final boolean isFirst) { + final String[] sortColumnNames = sortColumns.stream().map(sc -> { + descendingSortedFirstOrLastUnsupported(sc, isFirst); + return sc.column().name(); + }).toArray(String[]::new); + final ChunkSource.WithPrev inputSource; + if (sortColumnNames.length == 1) { + inputSource = table.getColumnSource(sortColumnNames[0]); + } else { + // Create a tuple source, because our underlying SSA does not handle multiple sort columns + inputSource = TupleSourceFactory.makeTupleSource( + Arrays.stream(sortColumnNames).map(table::getColumnSource).toArray(ColumnSource[]::new)); + } + // TODO-RWC: Move this helper here or to a new landing place + operators.add(SortedFirstOrLastByAggregationFactory.makeOperator(inputSource.getChunkType(), isFirst, + aggregations.size() > 1, MatchPair.fromPairs(resultPairs), table)); + inputColumnNames.add(sortColumnNames); + inputSources.add(inputSource); + } + + void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sortColumn, final boolean isFirst) { + if (sortColumn.order() == SortColumn.Order.ASCENDING) { + return; + } + throw new UnsupportedOperationException(String.format("%s does not support sort order in %s", + isFirst ? "SortedFirst" : "SortedLast", sortColumn)); + } + } + + // ----------------------------------------------------------------------------------------------------------------- + // Standard Aggregations + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an + * {@link AggregationContext} for standard aggregations. Accumulates state by visiting each aggregation. + */ + private class StandardConverter extends Converter { + + private StandardConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + super(table, groupByColumnNames); + } + + // ------------------------------------------------------------------------------------------------------------- + // Aggregation.Visitor + // ------------------------------------------------------------------------------------------------------------- + + @Override + public void visit(@NotNull final FirstRowKey firstRowKey) { + visitFirstOrLastAgg(true, firstRowKey.column().name()); + } + + @Override + public void visit(@NotNull final LastRowKey lastRowKey) { + visitFirstOrLastAgg(false, lastRowKey.column().name()); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -182,9 +327,7 @@ public void visit(@NotNull final AggSpecDistinct distinct) { @Override public void visit(@NotNull final AggSpecGroup group) { streamUnsupported("Group"); - operators.add(new GroupByChunkedOperator((QueryTable) table, true, MatchPair.fromPairs(resultPairs))); - inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); - inputSources.add(null); + addNoInputOperator(new GroupByChunkedOperator(table, true, MatchPair.fromPairs(resultPairs))); } @Override @@ -200,13 +343,11 @@ public void visit(@NotNull final AggSpecFirst first) { @Override public void visit(@NotNull final AggSpecFormula formula) { streamUnsupported("Formula"); - final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator((QueryTable) table, false, + final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, resultPairs.stream().map(pair -> MatchPair.of((Pair) pair.input())).toArray(MatchPair[]::new)); final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator(groupByChunkedOperator, true, formula.formula(), formula.formulaParam(), MatchPair.fromPairs(resultPairs)); - operators.add(formulaChunkedOperator); - inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); - inputSources.add(null); + addNoInputOperator(formulaChunkedOperator); } @Override @@ -277,230 +418,282 @@ public void visit(@NotNull final AggSpecWSum wSum) { public void visit(@NotNull final AggSpecVar var) { visitBasicAgg((t, n) -> getVarChunked(t, n, false, false)); } + } - private void visitBasicAgg(BiFunction, String, IterativeChunkedAggregationOperator> operatorFactory) { - for (final Pair pair : resultPairs) { - final String inputName = pair.input().name(); - final String resultName = pair.output().name(); - final ColumnSource rawInputSource = table.getColumnSource(inputName); - final Class type = rawInputSource.getType(); - final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + // ----------------------------------------------------------------------------------------------------------------- + // Rollup Unsupported Operations + // ----------------------------------------------------------------------------------------------------------------- - operators.add(operatorFactory.apply(type, resultName)); - inputColumnNames.add(new String[] {inputName}); - inputSources.add(inputSource); - } + private interface UnsupportedRollupAggregations extends RollupAggregation.Visitor, AggSpec.Visitor { + + @Override + @FinalDefault + default void visit(@NotNull final FirstRowKey firstRowKey) { + rollupUnsupported("FirstRowKey"); } - private void visitMinOrMaxAgg(final boolean isMin) { - for (final Pair pair : resultPairs) { - final String inputName = pair.input().name(); - final String resultName = pair.output().name(); - final ColumnSource rawInputSource = table.getColumnSource(inputName); - final Class type = rawInputSource.getType(); - final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + @Override + @FinalDefault + default void visit(@NotNull final LastRowKey lastRowKey) { + rollupUnsupported("LastRowKey"); + } - IntStream.range(0, inputSources.size()) - .filter(index -> (inputSources.get(index) == inputSource) - && (operators.get(index) instanceof SsmChunkedMinMaxOperator)) - .findFirst().ifPresentOrElse( - (final int priorMinMaxIndex) -> { - final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = - (SsmChunkedMinMaxOperator) operators.get(priorMinMaxIndex); - operators.add(ssmChunkedMinMaxOperator.makeSecondaryOperator(isMin, resultName)); - inputColumnNames.add(new String[] {inputName}); - inputSources.add(null); - }, - () -> { - operators.add(getMinMaxChunked(type, resultName, isMin, isAddOnly || isStream)); - inputColumnNames.add(new String[] {inputName}); - inputSources.add(inputSource); - }); - } + @Override + @FinalDefault + default void visit(@NotNull final AggSpecGroup group) { + rollupUnsupported("Group"); } - private void visitFirstOrLastAgg(final boolean isFirst, final String exposeRedirectionAs) { - if (exposeRedirectionAs != null) { - streamUnsupported((isFirst ? "First" : "Last") + - " with exposed row redirections (e.g. for rollup(), AggFirstRowKey, or AggLastRowKey)"); - } - final MatchPair[] resultMatchPairs = MatchPair.fromPairs(resultPairs); - if (table.isRefreshing()) { - if (isAddOnly) { - operators.add(new AddOnlyFirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, - exposeRedirectionAs)); - } else if (isStream) { - operators.add(isFirst - ? new StreamFirstChunkedOperator(resultMatchPairs, table) - : new StreamLastChunkedOperator(resultMatchPairs, table)); - } else { - if (trackedFirstOrLastIndex >= 0) { - operators.add(((FirstOrLastChunkedOperator) operators.get(trackedFirstOrLastIndex)) - .makeSecondaryOperator(isFirst, resultMatchPairs, table, exposeRedirectionAs)); - } else { - trackedFirstOrLastIndex = operators.size(); - operators.add(new FirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, - exposeRedirectionAs)); - } - } - } else { - operators.add(new StaticFirstOrLastChunkedOperator(isFirst, resultMatchPairs, table, - exposeRedirectionAs)); - } - inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); - inputSources.add(null); + @Override + @FinalDefault + default void visit(@NotNull final AggSpecFormula formula) { + rollupUnsupported("Formula"); } - private void streamUnsupported(@NotNull final String operationName) { - if (!isStream) { - return; - } - throw new UnsupportedOperationException(String.format( - "Stream tables do not support Agg%s; use StreamTableTools.streamToAppendOnlyTable to accumulate full history", - operationName)); + @Override + @FinalDefault + default void visit(@NotNull final AggSpecMedian median) { + rollupUnsupported("Median"); } - private void visitSortedFirstOrLastAgg(@NotNull final List sortColumns, final boolean isFirst) { - final String[] sortColumnNames = sortColumns.stream().map(sc -> { - descendingSortedFirstOrLastUnsupported(sc, isFirst); - return sc.column().name(); - }).toArray(String[]::new); - final ChunkSource.WithPrev inputSource; - if (sortColumnNames.length == 1) { - inputSource = table.getColumnSource(sortColumnNames[0]); - } else { - // Create a tuple source, because our underlying SSA does not handle multiple sort columns - inputSource = TupleSourceFactory.makeTupleSource( - Arrays.stream(sortColumnNames).map(table::getColumnSource).toArray(ColumnSource[]::new)); - } - // TODO-RWC: Move this helper here or to a new landing place - operators.add(SortedFirstOrLastByAggregationFactory.makeOperator(inputSource.getChunkType(), isFirst, - aggregations.size() > 1, MatchPair.fromPairs(resultPairs), table)); - inputColumnNames.add(sortColumnNames); - inputSources.add(inputSource); + @Override + @FinalDefault + default void visit(@NotNull final AggSpecPercentile pct) { + rollupUnsupported("Percentile"); } - private void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sortColumn, - final boolean isFirst) { - if (sortColumn.order() == SortColumn.Order.ASCENDING) { - return; - } - throw new UnsupportedOperationException(String.format("%s does not support sort order in %s", - isFirst ? "SortedFirst" : "SortedLast", sortColumn)); + @Override + @FinalDefault + default void visit(@NotNull final AggSpecWAvg wAvg) { + rollupUnsupported("WAvg"); } } + private static void rollupUnsupported(@NotNull final String operationName) { + throw new UnsupportedOperationException(String.format("Agg%s is not supported for rollup()", operationName)); + } + + // ----------------------------------------------------------------------------------------------------------------- + // Rollup Base-level Aggregations + // ----------------------------------------------------------------------------------------------------------------- + /** * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an * {@link AggregationContext} for rollup base aggregations. */ - private class RollupBaseConverter extends StandardConverter implements RollupAggregation.Visitor { + private class RollupBaseConverter extends Converter + implements RollupAggregation.Visitor, UnsupportedRollupAggregations { + + private boolean partitionFound; protected RollupBaseConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { super(table, groupByColumnNames); } + @Override + AggregationContext build() { + if (!partitionFound) { + transformers.add(new NoKeyLeafRollupAttributeSetter()); + } + return super.build(); + } + // ------------------------------------------------------------------------------------------------------------- // RollupAggregation.Visitor // ------------------------------------------------------------------------------------------------------------- @Override public void visit(@NotNull final NullColumns nullColumns) { - // TODO-RWC: This is not for base! - transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns()); + transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns())); } @Override public void visit(@NotNull final Partition partition) { + if (partition.includeConstituents()) { + streamUnsupported("Partition for rollup with constituents included"); + } + final QueryTable adjustedTable = partition.includeConstituents() + ? maybeCopyRlAttribute(table, table.updateView(ROLLUP_COLUMN + " = null")) + : table; + final PartitionByChunkedOperator.AttributeCopier copier = partition.includeConstituents() + ? RollupAttributeCopier.LEAF_WITHCONSTITUENTS_INSTANCE + : RollupAttributeCopier.DEFAULT_INSTANCE; + final PartitionByChunkedOperator partitionOperator = new PartitionByChunkedOperator(table, + adjustedTable, copier, Collections.emptyList(), groupByColumnNames); + + addNoInputOperator(partitionOperator); + transformers.add(makeRollupKeysTransformer(groupByColumnNames)); + transformers.add(new RollupTableMapAndReverseLookupAttributeSetter(partitionOperator, false, + partition.includeConstituents())); + + partitionFound = true; } - @Override - public void visit(@NotNull final Count count) { + // ------------------------------------------------------------------------------------------------------------- + // AggSpec.Visitor + // ------------------------------------------------------------------------------------------------------------- + @Override + public void visit(@NotNull final AggSpecAbsSum absSum) { + // Supported } @Override - public void visit(@NotNull final FirstRowKey firstRowKey) { + public void visit(@NotNull final AggSpecCountDistinct countDistinct) { + // Supported + } + @Override + public void visit(@NotNull final AggSpecDistinct distinct) { + // Supported } @Override - public void visit(@NotNull final LastRowKey lastRowKey) { + public void visit(@NotNull final AggSpecAvg avg) { + // Supported + } + @Override + public void visit(@NotNull final AggSpecFirst first) { + // Supported } @Override - public void visit(@NotNull final ColumnAggregation columnAgg) { + public void visit(@NotNull final AggSpecLast last) { + // Supported + } + @Override + public void visit(@NotNull final AggSpecMax max) { + // Supported } @Override - public void visit(@NotNull final ColumnAggregations columnAggs) { + public void visit(@NotNull final AggSpecMin min) { + // Supported + } + @Override + public void visit(@NotNull final AggSpecSortedFirst sortedFirst) { + // Supported } - // ------------------------------------------------------------------------------------------------------------- - // AggSpec.Visitor - // ------------------------------------------------------------------------------------------------------------- + @Override + public void visit(@NotNull final AggSpecSortedLast sortedLast) { + // Supported + } @Override - public void visit(@NotNull final AggSpecAbsSum absSum) { + public void visit(@NotNull final AggSpecStd std) { + // Supported + } + @Override + public void visit(@NotNull final AggSpecSum sum) { + // Supported } @Override - public void visit(@NotNull final AggSpecCountDistinct countDistinct) { + public void visit(@NotNull final AggSpecUnique unique) { + // Supported + } + @Override + public void visit(@NotNull final AggSpecWSum wSum) { + // Supported } @Override - public void visit(@NotNull final AggSpecDistinct distinct) { + public void visit(@NotNull final AggSpecVar var) { + // Supported + } + } + // ----------------------------------------------------------------------------------------------------------------- + // Rollup Reaggregated Aggregations + // ----------------------------------------------------------------------------------------------------------------- + + /** + * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an + * {@link AggregationContext} for rollup reaggregated (not base level) aggregations. + */ + private class RollupReaggregatedConverter extends Converter + implements RollupAggregation.Visitor, UnsupportedRollupAggregations { + + protected RollupReaggregatedConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + super(table, groupByColumnNames); } + // ------------------------------------------------------------------------------------------------------------- + // RollupAggregation.Visitor + // ------------------------------------------------------------------------------------------------------------- + @Override - public void visit(@NotNull final AggSpecGroup group) { + public void visit(@NotNull final NullColumns nullColumns) { + transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns())); + } + @Override + public void visit(@NotNull final Partition partition) { + if (partition.includeConstituents()) { + throw new IllegalArgumentException("Cannot include constituents for reaggregated rollup levels"); + } + + final List columnsToDrop = table.getDefinition().getColumnStream().map(ColumnDefinition::getName) + .filter(cn -> cn.endsWith(ROLLUP_COLUMN_SUFFIX)).collect(Collectors.toList()); + final QueryTable adjustedTable = columnsToDrop.isEmpty() + ? table + : maybeCopyRlAttribute(table, table.dropColumns(columnsToDrop)); + final PartitionByChunkedOperator partitionOperator = new PartitionByChunkedOperator(table, + adjustedTable, RollupAttributeCopier.DEFAULT_INSTANCE, Collections.emptyList(), groupByColumnNames); + + addNoInputOperator(partitionOperator); + transformers.add(makeRollupKeysTransformer(groupByColumnNames)); + transformers.add(new RollupTableMapAndReverseLookupAttributeSetter(partitionOperator, true, false)); } + // ------------------------------------------------------------------------------------------------------------- + // AggSpec.Visitor + // ------------------------------------------------------------------------------------------------------------- + @Override - public void visit(@NotNull final AggSpecAvg avg) { + public void visit(@NotNull final AggSpecAbsSum absSum) { } @Override - public void visit(@NotNull final AggSpecFirst first) { + public void visit(@NotNull final AggSpecCountDistinct countDistinct) { } @Override - public void visit(@NotNull final AggSpecFormula formula) { + public void visit(@NotNull final AggSpecDistinct distinct) { } @Override - public void visit(@NotNull final AggSpecLast last) { + public void visit(@NotNull final AggSpecAvg avg) { } @Override - public void visit(@NotNull final AggSpecMax max) { + public void visit(@NotNull final AggSpecFirst first) { } @Override - public void visit(@NotNull final AggSpecMedian median) { + public void visit(@NotNull final AggSpecLast last) { } @Override - public void visit(@NotNull final AggSpecMin min) { + public void visit(@NotNull final AggSpecMax max) { } @Override - public void visit(@NotNull final AggSpecPercentile pct) { + public void visit(@NotNull final AggSpecMin min) { } @@ -529,11 +722,6 @@ public void visit(@NotNull final AggSpecUnique unique) { } - @Override - public void visit(@NotNull final AggSpecWAvg wAvg) { - - } - @Override public void visit(@NotNull final AggSpecWSum wSum) { @@ -545,9 +733,115 @@ public void visit(@NotNull final AggSpecVar var) { } } + // ----------------------------------------------------------------------------------------------------------------- + // Helpers + // ----------------------------------------------------------------------------------------------------------------- + private static ColumnSource maybeReinterpretDateTimeAsLong(@NotNull final ColumnSource inputSource) { return inputSource.getType() == DateTime.class ? ReinterpretUtils.dateTimeToLongSource(inputSource) : inputSource; } + + @NotNull + private static String makeRedirectionName(IterativeIndexSpec inputAggregationStateFactory) { + return IterativeIndexSpec.ROW_REDIRECTION_PREFIX + inputAggregationStateFactory.rollupColumnIdentifier + + RollupConstants.ROLLUP_COLUMN_SUFFIX; + } + + private static QueryTable maybeCopyRlAttribute(@NotNull final Table parent, @NotNull final Table child) { + if (child != parent && parent.hasAttribute(REVERSE_LOOKUP_ATTRIBUTE)) { + child.setAttribute(REVERSE_LOOKUP_ATTRIBUTE, parent.getAttribute(REVERSE_LOOKUP_ATTRIBUTE)); + } + return (QueryTable) child; + } + + private static AggregationContextTransformer makeRollupKeysTransformer(@NotNull final String[] groupByColumnNames) { + if (groupByColumnNames.length == 0) { + return new StaticColumnSourceTransformer(RollupInfo.ROLLUP_COLUMN, + new SingleValueObjectColumnSource<>(SmartKey.EMPTY)); + } + if (groupByColumnNames.length == 1) { + return new RollupKeyColumnDuplicationTransformer(groupByColumnNames[0]); + } + return new RollupSmartKeyColumnDuplicationTransformer(groupByColumnNames); + } + + private static class RollupTableMapAndReverseLookupAttributeSetter implements AggregationContextTransformer { + + private final PartitionByChunkedOperator partitionOperator; + private final boolean reaggregated; + private final boolean includeConstituents; + private ReverseLookup reverseLookup; + + private RollupTableMapAndReverseLookupAttributeSetter( + @NotNull final PartitionByChunkedOperator partitionOperator, + final boolean reaggregated, + final boolean includeConstituents) { + this.partitionOperator = partitionOperator; + this.reaggregated = reaggregated; + this.includeConstituents = includeConstituents; + } + + @Override + public QueryTable transformResult(@NotNull final QueryTable table) { + table.setAttribute(QueryTable.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, partitionOperator.getTableMap()); + if (reaggregated || includeConstituents) { + table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, reverseLookup); + } else { + setRollupLeafAttributes(table); + } + return table; + } + + @Override + public void setReverseLookupFunction(ToIntFunction reverseLookup) { + this.reverseLookup = new ReverseLookupAdapter(reverseLookup); + } + + private static class ReverseLookupAdapter implements ReverseLookup { + + private static final String[] KEY_COLUMN_NAMES = new String[] {ROLLUP_COLUMN}; + + private final ToIntFunction reverseLookup; + + public ReverseLookupAdapter(@NotNull final ToIntFunction reverseLookup) { + this.reverseLookup = reverseLookup; + } + + @Override + public long get(final Object key) { + return reverseLookup.applyAsInt(key); + } + + @Override + public long getPrev(final Object key) { + return get(key); + } + + @Override + public long getNoEntryValue() { + return -1; + } + + @Override + public String[] getKeyColumns() { + return KEY_COLUMN_NAMES; + } + } + } + + private static class NoKeyLeafRollupAttributeSetter implements AggregationContextTransformer { + @Override + public QueryTable transformResult(@NotNull final QueryTable table) { + setRollupLeafAttributes(table); + return table; + } + } + + private static void setRollupLeafAttributes(@NotNull final QueryTable table) { + table.setAttribute(Table.ROLLUP_LEAF_ATTRIBUTE, RollupInfo.LeafType.Normal); + table.setAttribute(Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, EmptyTableMap.INSTANCE); + table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, ReverseLookup.NULL); + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java new file mode 100644 index 00000000000..18f59a90b1b --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java @@ -0,0 +1,19 @@ +package io.deephaven.engine.table.impl.by; + +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.Pair; +import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.BaseTable; +import org.jetbrains.annotations.NotNull; + +import java.util.ArrayList; +import java.util.List; + +/** + * @author rcaudy + * @since 2022-01-21 + */ + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedAvgOperator.java index 3b132e32a62..2cd0ba48535 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedAvgOperator.java @@ -18,7 +18,7 @@ import java.util.LinkedHashMap; import java.util.Map; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; class BigDecimalChunkedAvgOperator implements IterativeChunkedAggregationOperator { private final String name; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedVarOperator.java index 18bc284807b..98794c713ee 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigDecimalChunkedVarOperator.java @@ -21,7 +21,7 @@ import java.util.LinkedHashMap; import java.util.Map; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedAvgOperator.java index c32b598fe3b..7a1bc504f46 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedAvgOperator.java @@ -19,7 +19,8 @@ import java.util.LinkedHashMap; import java.util.Map; -import static io.deephaven.engine.table.impl.by.AggregationFactory.ROLLUP_COLUMN_SUFFIX; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_COLUMN_SUFFIX; class BigIntegerChunkedAvgOperator implements IterativeChunkedAggregationOperator { private final String name; @@ -117,8 +118,8 @@ public void ensureCapacity(long tableSize) { if (exposeInternalColumns) { final Map> results = new LinkedHashMap<>(); results.put(name, resultColumn); - results.put(name + AggregationFactory.ROLLUP_RUNNING_SUM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, runningSum); - results.put(name + AggregationFactory.ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, nonNullCount.getColumnSource()); + results.put(name + ROLLUP_RUNNING_SUM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, runningSum); + results.put(name + ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, nonNullCount.getColumnSource()); return results; } else { return Collections.singletonMap(name, resultColumn); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedVarOperator.java index d2b6863eafc..fc578f7328f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BigIntegerChunkedVarOperator.java @@ -22,7 +22,7 @@ import java.util.LinkedHashMap; import java.util.Map; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java index 5d9f42d654a..b2e50e12729 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedAvgOperator.java @@ -23,9 +23,9 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative average operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java index 4dc9b0d4859..bd9e9f4fd98 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteChunkedVarOperator.java @@ -23,8 +23,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java index c8cca07654b..e40a38e410c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedAvgOperator.java @@ -18,9 +18,9 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative average operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java index 2c8b8e0dd6c..b8dbb7292ba 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CharChunkedVarOperator.java @@ -18,8 +18,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java index 7267290c003..6e2b57c050e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedAvgOperator.java @@ -22,8 +22,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; class DoubleChunkedAvgOperator extends FpChunkedNonNormalCounter implements IterativeChunkedAggregationOperator { private final String name; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java index 521650f585d..0a0efa4eaa3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleChunkedVarOperator.java @@ -24,7 +24,7 @@ import java.util.LinkedHashMap; import java.util.Map; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java index a5238bc5d55..6ee100c8adf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java @@ -34,7 +34,7 @@ ReaggregatableStatefactory forRollup() { @Override ReaggregatableStatefactory rollupFactory() { return new SortedFirstOrLastByFactoryImpl(true, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + AggregationFactory.ROLLUP_COLUMN_SUFFIX); + ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java index 443b7620694..1db3f7da4c3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedAvgOperator.java @@ -17,8 +17,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; class FloatChunkedAvgOperator extends FpChunkedNonNormalCounter implements IterativeChunkedAggregationOperator { private final String name; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java index e65b7df2bb8..c737094ac10 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatChunkedVarOperator.java @@ -19,7 +19,7 @@ import java.util.LinkedHashMap; import java.util.Map; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java index ebe06b02268..efa3f7c927c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java @@ -19,7 +19,7 @@ import java.util.Map; import java.util.function.Supplier; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; abstract class FpChunkedNonNormalCounter { // individual state values for nan, positive infinity, and negative infinity diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java index 5cd48bb8e5d..892c60a5a67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedAvgOperator.java @@ -23,9 +23,9 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative average operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java index 67bce47e95a..17e923de066 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntChunkedVarOperator.java @@ -23,8 +23,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java index 3a5c5305611..59594407d66 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java @@ -15,6 +15,8 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; + public abstract class IterativeIndexSpec extends ReaggregatableStatefactory { static final String ROW_REDIRECTION_PREFIX = "RowRedirection_"; private final Map nameToDestColumns = new LinkedHashMap<>(); @@ -41,7 +43,7 @@ public abstract class IterativeIndexSpec extends ReaggregatableStatefactory { @NotNull private String getRedirectionName() { - return ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + AggregationFactory.ROLLUP_COLUMN_SUFFIX; + return ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + ROLLUP_COLUMN_SUFFIX; } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java index 8827ff74aea..51c5e217419 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java @@ -32,7 +32,7 @@ ReaggregatableStatefactory forRollup() { @Override ReaggregatableStatefactory rollupFactory() { return new SortedFirstOrLastByFactoryImpl(false, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + AggregationFactory.ROLLUP_COLUMN_SUFFIX); + ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java index 6ff4b92c0dd..17c33a0c8f2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedAvgOperator.java @@ -23,9 +23,9 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative average operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java index 519ebb646a5..11352eab084 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongChunkedVarOperator.java @@ -23,8 +23,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java index d0de6d59bd1..3e69e16996d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedAvgOperator.java @@ -23,9 +23,9 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusLong; import static io.deephaven.engine.util.NullSafeAddition.minusLong; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative average operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java index c2a70a048e5..952f8a7b85b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortChunkedVarOperator.java @@ -23,8 +23,8 @@ import java.util.LinkedHashMap; import java.util.Map; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.engine.util.NullSafeAddition.plusDouble; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; /** * Iterative variance operator. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java index 942f54150d5..8bede585dd1 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java @@ -32,7 +32,7 @@ ReaggregatableStatefactory forRollup() { @Override ReaggregatableStatefactory rollupFactory() { return new SortedFirstOrLastByFactoryImpl(true, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + AggregationFactory.ROLLUP_COLUMN_SUFFIX); + ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java index 64f1c3f322f..6f003dc618c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java @@ -32,7 +32,7 @@ ReaggregatableStatefactory forRollup() { @Override ReaggregatableStatefactory rollupFactory() { return new SortedFirstOrLastByFactoryImpl(false, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + AggregationFactory.ROLLUP_COLUMN_SUFFIX); + ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.java index 5309b6cc075..8d02b4a9266 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -261,7 +261,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.java index 19ad706ef3d..3ca7ae3da96 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ByteSsmBackedSource; @@ -513,7 +513,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.java index 0ea3c424b93..755c54c6ef5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.java @@ -8,8 +8,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -256,7 +256,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.java index 6e12553d4c4..0e409c7c6d7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.java @@ -8,8 +8,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.CharSsmBackedSource; @@ -508,7 +508,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.java index 4c42e6db042..f6b9a0162af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -261,7 +261,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.java index e205127dd19..dfb431b78f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DoubleSsmBackedSource; @@ -513,7 +513,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.java index e125294d4d4..09858d077e7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -261,7 +261,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.java index 1f6a5851e0b..d94c58f71a3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.FloatSsmBackedSource; @@ -513,7 +513,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.java index 224f6104ff0..06c8352ec96 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -261,7 +261,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.java index 1b8adfce8c5..35cfc062d4a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.IntSsmBackedSource; @@ -513,7 +513,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.java index 4148e6f3884..4231aab3011 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -261,7 +261,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.java index 0d843e850d1..c1d1fa359ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.LongSsmBackedSource; @@ -513,7 +513,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.java index 765d77839e5..8abef96a750 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -262,7 +262,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.java index 2bd877956e4..f2d2edce4a6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ObjectSsmBackedSource; @@ -514,7 +514,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.java index d5d0f890dc9..96174ec9432 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.util.QueryConstants; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; @@ -261,7 +261,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.java index 805c80c0ffc..6ab51357df3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ShortSsmBackedSource; @@ -513,7 +513,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, resultColumn); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.java index 8d7f48e5b5a..5d3d7ea8c7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ByteSsmBackedSource; @@ -264,7 +264,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.java index 2b5d4178470..c75d94a8f21 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ByteSsmBackedSource; @@ -491,7 +491,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.java index 9ace9bb6b64..0743938f623 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.java @@ -8,8 +8,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.CharSsmBackedSource; @@ -259,7 +259,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.java index d220398451e..6266b1dc4cf 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.java @@ -8,8 +8,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.CharSsmBackedSource; @@ -486,7 +486,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.java index fcc328b2dde..1c852b70577 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DoubleSsmBackedSource; @@ -264,7 +264,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.java index f0954fea0e6..1b0e5c084ea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DoubleSsmBackedSource; @@ -491,7 +491,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.java index 6715499600a..d398c815a03 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.FloatSsmBackedSource; @@ -264,7 +264,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.java index 7030608d84a..4d9115904af 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.FloatSsmBackedSource; @@ -491,7 +491,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.java index 0293f408aa6..18189c6ed25 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.IntSsmBackedSource; @@ -264,7 +264,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.java index 591da8250ad..3f345a5fd18 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.IntSsmBackedSource; @@ -491,7 +491,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java index dc054233950..82394d2d9ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java @@ -9,7 +9,7 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct; -import io.deephaven.engine.table.impl.sources.BoxedColumnSource; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.time.DateTime; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; @@ -18,7 +18,6 @@ import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.LongSsmBackedSource; @@ -273,7 +272,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java index c3012a07827..8863531717d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java @@ -9,7 +9,7 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct; -import io.deephaven.engine.table.impl.sources.BoxedColumnSource; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.time.DateTime; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; @@ -18,7 +18,6 @@ import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.LongSsmBackedSource; @@ -500,7 +499,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.java index a798c7c7d4b..92fc830ac85 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ObjectSsmBackedSource; @@ -265,7 +265,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.java index 74358a2d1d2..865c7bb3d50 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ObjectSsmBackedSource; @@ -492,7 +492,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.java index 83c1e9825f9..b10551752ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ShortSsmBackedSource; @@ -264,7 +264,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.java index 3af904d971d..8bee2f7f12f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ShortSsmBackedSource; @@ -491,7 +491,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, internalResult.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java index 19681b8437f..8b8b4673800 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ByteSsmBackedSource; @@ -272,7 +272,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java index f1681facc99..eac1499810a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ByteSsmBackedSource; @@ -517,7 +517,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java index 13a3d556be1..da90680b086 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.java @@ -8,8 +8,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.CharSsmBackedSource; @@ -267,7 +267,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java index 1899fe7710e..004241e6ea9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.java @@ -8,8 +8,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.CharSsmBackedSource; @@ -512,7 +512,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java index 2d37fb196de..09620d07ca3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DoubleSsmBackedSource; @@ -272,7 +272,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java index 78965ff35d3..c47d7623379 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DoubleSsmBackedSource; @@ -517,7 +517,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java index 8d026e00a7b..bc498a5a106 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.FloatSsmBackedSource; @@ -272,7 +272,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java index 72f320812f7..7fc796f6cb8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.FloatSsmBackedSource; @@ -517,7 +517,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java index 8de8237b30a..8682821e7b6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.IntSsmBackedSource; @@ -272,7 +272,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java index 35ee6f29dd5..6ba72ba1edd 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.IntSsmBackedSource; @@ -517,7 +517,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java index b88abee247a..81d463402ce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java @@ -9,16 +9,15 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.unique; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.table.impl.sources.BoxedColumnSource; import io.deephaven.time.DateTime; -import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.LongSsmBackedSource; @@ -281,7 +280,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java index a74bedee32a..381675610de 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java @@ -9,16 +9,15 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.unique; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.table.impl.sources.BoxedColumnSource; import io.deephaven.time.DateTime; -import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.LongSsmBackedSource; @@ -526,7 +525,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java index cb1639300d5..49f93413a0b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ObjectSsmBackedSource; @@ -273,7 +273,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java index c69c1bf79be..c74e27ff2ef 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ObjectSsmBackedSource; @@ -518,7 +518,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java index 97cd7a94cb3..fc12a6e693e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ShortSsmBackedSource; @@ -272,7 +272,7 @@ public void ensureCapacity(long tableSize) { if(exposeInternal) { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java index 9ea22ae7fbe..fe24f073940 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.java @@ -13,8 +13,8 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; import io.deephaven.engine.table.impl.by.ssmcountdistinct.ShortSsmBackedSource; @@ -517,7 +517,7 @@ public void ensureCapacity(long tableSize) { public Map> getResultColumns() { final Map> columns = new LinkedHashMap<>(); columns.put(name, externalResult); - columns.put(name + AggregationFactory.ROLLUP_DISTINCT_SSM_COLUMN_ID + AggregationFactory.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); + columns.put(name + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX, ssms.getUnderlyingSource()); return columns; } From 8a2322e4f07afc0ad33edc00540156e3cbd42fa5 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 25 Jan 2022 09:41:56 -0500 Subject: [PATCH 13/44] Plumbed most of rollup base level. --- .../table/impl/by/AggregationProcessor.java | 69 +++++++++++++------ 1 file changed, 47 insertions(+), 22 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 6cedb382164..9335541ebb3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -73,7 +73,9 @@ private AggregationProcessor( // ----------------------------------------------------------------------------------------------------------------- @Override - public AggregationContext makeAggregationContext(@NotNull Table table, @NotNull String... groupByColumnNames) { + public AggregationContext makeAggregationContext(@NotNull final Table table, + final int rollupLevel, + @NotNull final String... groupByColumnNames) { switch (type) { case STANDARD: return new StandardConverter(table, groupByColumnNames).build(); @@ -148,15 +150,6 @@ void streamUnsupported(@NotNull final String operationName) { // Partial Aggregation.Visitor (for cases common to all types) // ------------------------------------------------------------------------------------------------------------- - @Override - public final void visit(@NotNull final Count count) { - addNoInputOperator(new CountAggregationOperator(count.column().name())); - } - - // ------------------------------------------------------------------------------------------------------------- - // Partial AggSpec.Visitor (for cases common to all types) - // ------------------------------------------------------------------------------------------------------------- - @Override public final void visit(@NotNull final ColumnAggregation columnAgg) { resultPairs = List.of(columnAgg.pair()); @@ -171,6 +164,12 @@ public final void visit(@NotNull final ColumnAggregations columnAggs) { resultPairs = List.of(); } + // ------------------------------------------------------------------------------------------------------------- + // Partial AggSpec.Visitor (for cases common to all types) + // ------------------------------------------------------------------------------------------------------------- + + // THIS SPACE INTENTIONALLY LEFT BLANK + // ------------------------------------------------------------------------------------------------------------- // Helpers for visitors // ------------------------------------------------------------------------------------------------------------- @@ -294,6 +293,11 @@ private StandardConverter(@NotNull final Table table, @NotNull final String... g // Aggregation.Visitor // ------------------------------------------------------------------------------------------------------------- + @Override + public final void visit(@NotNull final Count count) { + addNoInputOperator(new CountAggregationOperator(count.column().name())); + } + @Override public void visit(@NotNull final FirstRowKey firstRowKey) { visitFirstOrLastAgg(true, firstRowKey.column().name()); @@ -426,6 +430,10 @@ public void visit(@NotNull final AggSpecVar var) { private interface UnsupportedRollupAggregations extends RollupAggregation.Visitor, AggSpec.Visitor { + // ------------------------------------------------------------------------------------------------------------- + // RollupAggregation.Visitor for unsupported aggregations + // ------------------------------------------------------------------------------------------------------------- + @Override @FinalDefault default void visit(@NotNull final FirstRowKey firstRowKey) { @@ -438,6 +446,10 @@ default void visit(@NotNull final LastRowKey lastRowKey) { rollupUnsupported("LastRowKey"); } + // ------------------------------------------------------------------------------------------------------------- + // AggSpec.Visitor for unsupported column aggregation specs + // ------------------------------------------------------------------------------------------------------------- + @Override @FinalDefault default void visit(@NotNull final AggSpecGroup group) { @@ -502,6 +514,11 @@ AggregationContext build() { // RollupAggregation.Visitor // ------------------------------------------------------------------------------------------------------------- + @Override + public final void visit(@NotNull final Count count) { + addNoInputOperator(new CountAggregationOperator(count.column().name())); + } + @Override public void visit(@NotNull final NullColumns nullColumns) { transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns())); @@ -536,22 +553,22 @@ public void visit(@NotNull final Partition partition) { @Override public void visit(@NotNull final AggSpecAbsSum absSum) { - // Supported + visitBasicAgg(IterativeOperatorSpec::getAbsSumChunked); } @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - // Supported + visitBasicAgg((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), true, false)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - // Supported + visitBasicAgg((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), true, false)); } @Override public void visit(@NotNull final AggSpecAvg avg) { - // Supported + visitBasicAgg((t, n) -> getAvgChunked(t, n, true)); } @Override @@ -566,12 +583,12 @@ public void visit(@NotNull final AggSpecLast last) { @Override public void visit(@NotNull final AggSpecMax max) { - // Supported + visitMinOrMaxAgg(false); } @Override public void visit(@NotNull final AggSpecMin min) { - // Supported + visitMinOrMaxAgg(true); } @Override @@ -586,27 +603,29 @@ public void visit(@NotNull final AggSpecSortedLast sortedLast) { @Override public void visit(@NotNull final AggSpecStd std) { - // Supported + visitBasicAgg((t, n) -> getVarChunked(t, n, true, true)); } @Override public void visit(@NotNull final AggSpecSum sum) { - // Supported + visitBasicAgg(IterativeOperatorSpec::getSumChunked); } @Override public void visit(@NotNull final AggSpecUnique unique) { - // Supported + visitBasicAgg((t, n) -> getUniqueChunked(t, n, + unique.includeNulls(), null, unique.nonUniqueSentinel(), true, false)); } @Override public void visit(@NotNull final AggSpecWSum wSum) { - // Supported + WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, wSum.weight().name(), true, + MatchPair.fromPairs(resultPairs), operators, inputColumnNames, inputSources); } @Override public void visit(@NotNull final AggSpecVar var) { - // Supported + visitBasicAgg((t, n) -> getVarChunked(t, n, false, true)); } } @@ -629,6 +648,12 @@ protected RollupReaggregatedConverter(@NotNull final Table table, @NotNull final // RollupAggregation.Visitor // ------------------------------------------------------------------------------------------------------------- + @Override + public final void visit(@NotNull final Count count) { + // TODO-RWC: Re-agg as sum + addNoInputOperator(new CountAggregationOperator(count.column().name())); + } + @Override public void visit(@NotNull final NullColumns nullColumns) { transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns())); @@ -724,7 +749,7 @@ public void visit(@NotNull final AggSpecUnique unique) { @Override public void visit(@NotNull final AggSpecWSum wSum) { - + // Re-agg as sum } @Override From 21088137ad73f153e602d7c9dffe5a75554c48d5 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 25 Jan 2022 23:09:51 -0500 Subject: [PATCH 14/44] AggregationProcessor handles all cases now --- .../table/impl/by/AggregationFactory.java | 6 +- .../table/impl/by/AggregationProcessor.java | 384 +++++++++++++----- .../impl/by/FpChunkedNonNormalCounter.java | 12 +- 3 files changed, 293 insertions(+), 109 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java index 9dcc35f92f0..fd6e9132d12 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java @@ -560,11 +560,11 @@ public AggregationContextFactory makeAggregationContextFactory() { final String nonNullName = mp.leftColumn() + RollupConstants.ROLLUP_NONNULL_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String nanName = - mp.leftColumn() + RollupConstants.ROLLUP_NAN_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_NAN_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String picName = - mp.leftColumn() + RollupConstants.ROLLUP_PIC_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_PI_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final String nicName = - mp.leftColumn() + RollupConstants.ROLLUP_NIC_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; + mp.leftColumn() + RollupConstants.ROLLUP_NI_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; final boolean isFloatingPoint = table.hasColumns(nanName); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 9335541ebb3..fdf657d75b3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -14,21 +14,28 @@ import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.sources.SingleValueObjectColumnSource; +import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; import io.deephaven.time.DateTime; +import io.deephaven.util.FunctionalInterfaces.TriFunction; import io.deephaven.util.annotations.FinalDefault; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; +import java.math.BigDecimal; +import java.math.BigInteger; import java.util.*; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.ToIntFunction; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; import static io.deephaven.engine.table.Table.REVERSE_LOOKUP_ATTRIBUTE; import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; import static io.deephaven.engine.table.impl.by.IterativeOperatorSpec.*; -import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_COLUMN_SUFFIX; +import static io.deephaven.engine.table.impl.by.RollupConstants.*; /** * Conversion tool to generate an {@link AggregationContextFactory} for a collection of {@link Aggregation @@ -74,8 +81,7 @@ private AggregationProcessor( @Override public AggregationContext makeAggregationContext(@NotNull final Table table, - final int rollupLevel, - @NotNull final String... groupByColumnNames) { + @NotNull final String... groupByColumnNames) { switch (type) { case STANDARD: return new StandardConverter(table, groupByColumnNames).build(); @@ -131,12 +137,6 @@ AggregationContext build() { transformers.toArray(AggregationContextTransformer[]::new)); } - final void addNoInputOperator(@NotNull final IterativeChunkedAggregationOperator operator) { - operators.add(operator); - inputColumnNames.add(ZERO_LENGTH_STRING_ARRAY); - inputSources.add(null); - } - void streamUnsupported(@NotNull final String operationName) { if (!isStream) { return; @@ -174,7 +174,20 @@ public final void visit(@NotNull final ColumnAggregations columnAggs) { // Helpers for visitors // ------------------------------------------------------------------------------------------------------------- - void visitBasicAgg(BiFunction, String, IterativeChunkedAggregationOperator> operatorFactory) { + void addNoInputOperator(@NotNull final IterativeChunkedAggregationOperator operator) { + addOperator(operator, null, ZERO_LENGTH_STRING_ARRAY); + } + + final void addOperator(@NotNull final IterativeChunkedAggregationOperator operator, + @Nullable final ChunkSource.WithPrev inputSource, + @NotNull final String... inputColumnNames) { + operators.add(operator); + this.inputColumnNames.add(inputColumnNames); + inputSources.add(inputSource); + } + + final void addBasicOperators( + BiFunction, String, IterativeChunkedAggregationOperator> operatorFactory) { for (final Pair pair : resultPairs) { final String inputName = pair.input().name(); final String resultName = pair.output().name(); @@ -182,40 +195,42 @@ void visitBasicAgg(BiFunction, String, IterativeChunkedAggregationOpera final Class type = rawInputSource.getType(); final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); - operators.add(operatorFactory.apply(type, resultName)); - inputColumnNames.add(new String[] {inputName}); - inputSources.add(inputSource); + addOperator(operatorFactory.apply(type, resultName), inputSource, inputName); } } - void visitMinOrMaxAgg(final boolean isMin) { + final void addMinOrMaxOperators(final boolean isMin) { for (final Pair pair : resultPairs) { final String inputName = pair.input().name(); final String resultName = pair.output().name(); - final ColumnSource rawInputSource = table.getColumnSource(inputName); - final Class type = rawInputSource.getType(); - final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); - IntStream.range(0, inputSources.size()) - .filter(index -> (inputSources.get(index) == inputSource) - && (operators.get(index) instanceof SsmChunkedMinMaxOperator)) - .findFirst().ifPresentOrElse( - (final int priorMinMaxIndex) -> { - final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = - (SsmChunkedMinMaxOperator) operators.get(priorMinMaxIndex); - operators.add(ssmChunkedMinMaxOperator.makeSecondaryOperator(isMin, resultName)); - inputColumnNames.add(new String[] {inputName}); - inputSources.add(null); - }, - () -> { - operators.add(getMinMaxChunked(type, resultName, isMin, isAddOnly || isStream)); - inputColumnNames.add(new String[] {inputName}); - inputSources.add(inputSource); - }); + addMinOrMaxOperator(isMin, inputName, resultName); } } - void visitFirstOrLastAgg(final boolean isFirst, final String exposeRedirectionAs) { + final void addMinOrMaxOperator(final boolean isMin, @NotNull final String inputName, + @NotNull final String resultName) { + final ColumnSource rawInputSource = table.getColumnSource(inputName); + final Class type = rawInputSource.getType(); + final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); + + IntStream.range(0, inputSources.size()) + .filter(index -> (inputSources.get(index) == inputSource) + && (operators.get(index) instanceof SsmChunkedMinMaxOperator)) + .findFirst().ifPresentOrElse( + (final int priorMinMaxIndex) -> { + final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = + (SsmChunkedMinMaxOperator) operators.get(priorMinMaxIndex); + addOperator( + ssmChunkedMinMaxOperator.makeSecondaryOperator(isMin, resultName), + null, inputName); + }, + () -> addOperator( + getMinMaxChunked(type, resultName, isMin, isAddOnly || isStream), + inputSource, inputName)); + } + + final void addFirstOrLastOperators(final boolean isFirst, final String exposeRedirectionAs) { if (exposeRedirectionAs != null) { streamUnsupported((isFirst ? "First" : "Last") + " with exposed row redirections (e.g. for rollup(), AggFirstRowKey, or AggLastRowKey)"); @@ -246,7 +261,7 @@ void visitFirstOrLastAgg(final boolean isFirst, final String exposeRedirectionAs addNoInputOperator(operator); } - void visitSortedFirstOrLastAgg(@NotNull final List sortColumns, final boolean isFirst) { + final void addSortedFirstOrLastOperator(@NotNull final List sortColumns, final boolean isFirst) { final String[] sortColumnNames = sortColumns.stream().map(sc -> { descendingSortedFirstOrLastUnsupported(sc, isFirst); return sc.column().name(); @@ -260,13 +275,12 @@ void visitSortedFirstOrLastAgg(@NotNull final List sortColumns, fina Arrays.stream(sortColumnNames).map(table::getColumnSource).toArray(ColumnSource[]::new)); } // TODO-RWC: Move this helper here or to a new landing place - operators.add(SortedFirstOrLastByAggregationFactory.makeOperator(inputSource.getChunkType(), isFirst, - aggregations.size() > 1, MatchPair.fromPairs(resultPairs), table)); - inputColumnNames.add(sortColumnNames); - inputSources.add(inputSource); + addOperator(SortedFirstOrLastByAggregationFactory.makeOperator(inputSource.getChunkType(), isFirst, + aggregations.size() > 1, MatchPair.fromPairs(resultPairs), table), + inputSource, sortColumnNames); } - void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sortColumn, final boolean isFirst) { + final void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sortColumn, final boolean isFirst) { if (sortColumn.order() == SortColumn.Order.ASCENDING) { return; } @@ -283,7 +297,7 @@ void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sortColumn * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an * {@link AggregationContext} for standard aggregations. Accumulates state by visiting each aggregation. */ - private class StandardConverter extends Converter { + private final class StandardConverter extends Converter { private StandardConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { super(table, groupByColumnNames); @@ -294,18 +308,18 @@ private StandardConverter(@NotNull final Table table, @NotNull final String... g // ------------------------------------------------------------------------------------------------------------- @Override - public final void visit(@NotNull final Count count) { + public void visit(@NotNull final Count count) { addNoInputOperator(new CountAggregationOperator(count.column().name())); } @Override public void visit(@NotNull final FirstRowKey firstRowKey) { - visitFirstOrLastAgg(true, firstRowKey.column().name()); + addFirstOrLastOperators(true, firstRowKey.column().name()); } @Override public void visit(@NotNull final LastRowKey lastRowKey) { - visitFirstOrLastAgg(false, lastRowKey.column().name()); + addFirstOrLastOperators(false, lastRowKey.column().name()); } // ------------------------------------------------------------------------------------------------------------- @@ -315,17 +329,17 @@ public void visit(@NotNull final LastRowKey lastRowKey) { @Override public void visit(@NotNull final AggSpecAbsSum absSum) { // TODO-RWC: Move this helper and its friends here or to a new landing place - visitBasicAgg(IterativeOperatorSpec::getAbsSumChunked); + addBasicOperators(IterativeOperatorSpec::getAbsSumChunked); } @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - visitBasicAgg((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), false, false)); + addBasicOperators((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), false, false)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - visitBasicAgg((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), false, false)); + addBasicOperators((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), false, false)); } @Override @@ -336,12 +350,12 @@ public void visit(@NotNull final AggSpecGroup group) { @Override public void visit(@NotNull final AggSpecAvg avg) { - visitBasicAgg((t, n) -> getAvgChunked(t, n, false)); + addBasicOperators((t, n) -> getAvgChunked(t, n, false)); } @Override public void visit(@NotNull final AggSpecFirst first) { - visitFirstOrLastAgg(true, null); + addFirstOrLastOperators(true, null); } @Override @@ -356,52 +370,52 @@ public void visit(@NotNull final AggSpecFormula formula) { @Override public void visit(@NotNull final AggSpecLast last) { - visitFirstOrLastAgg(false, null); + addFirstOrLastOperators(false, null); } @Override public void visit(@NotNull final AggSpecMax max) { - visitMinOrMaxAgg(false); + addMinOrMaxOperators(false); } @Override public void visit(@NotNull final AggSpecMedian median) { - visitBasicAgg((t, n) -> getPercentileChunked(t, n, 0.50d, median.averageMedian())); + addBasicOperators((t, n) -> getPercentileChunked(t, n, 0.50d, median.averageMedian())); } @Override public void visit(@NotNull final AggSpecMin min) { - visitMinOrMaxAgg(true); + addMinOrMaxOperators(true); } @Override public void visit(@NotNull final AggSpecPercentile pct) { - visitBasicAgg((t, n) -> getPercentileChunked(t, n, pct.percentile(), pct.averageMedian())); + addBasicOperators((t, n) -> getPercentileChunked(t, n, pct.percentile(), pct.averageMedian())); } @Override public void visit(@NotNull final AggSpecSortedFirst sortedFirst) { - visitSortedFirstOrLastAgg(sortedFirst.columns(), true); + addSortedFirstOrLastOperator(sortedFirst.columns(), true); } @Override public void visit(@NotNull final AggSpecSortedLast sortedLast) { - visitSortedFirstOrLastAgg(sortedLast.columns(), false); + addSortedFirstOrLastOperator(sortedLast.columns(), false); } @Override public void visit(@NotNull final AggSpecStd std) { - visitBasicAgg((t, n) -> getVarChunked(t, n, true, false)); + addBasicOperators((t, n) -> getVarChunked(t, n, true, false)); } @Override public void visit(@NotNull final AggSpecSum sum) { - visitBasicAgg(IterativeOperatorSpec::getSumChunked); + addBasicOperators(IterativeOperatorSpec::getSumChunked); } @Override public void visit(@NotNull final AggSpecUnique unique) { - visitBasicAgg((t, n) -> getUniqueChunked(t, n, + addBasicOperators((t, n) -> getUniqueChunked(t, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), false, false)); } @@ -420,7 +434,7 @@ public void visit(@NotNull final AggSpecWSum wSum) { @Override public void visit(@NotNull final AggSpecVar var) { - visitBasicAgg((t, n) -> getVarChunked(t, n, false, false)); + addBasicOperators((t, n) -> getVarChunked(t, n, false, false)); } } @@ -493,12 +507,13 @@ private static void rollupUnsupported(@NotNull final String operationName) { * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an * {@link AggregationContext} for rollup base aggregations. */ - private class RollupBaseConverter extends Converter + private final class RollupBaseConverter extends Converter implements RollupAggregation.Visitor, UnsupportedRollupAggregations { private boolean partitionFound; + private int nextColumnIdentifier = 0; - protected RollupBaseConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + private RollupBaseConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { super(table, groupByColumnNames); } @@ -515,7 +530,7 @@ AggregationContext build() { // ------------------------------------------------------------------------------------------------------------- @Override - public final void visit(@NotNull final Count count) { + public void visit(@NotNull final Count count) { addNoInputOperator(new CountAggregationOperator(count.column().name())); } @@ -553,67 +568,67 @@ public void visit(@NotNull final Partition partition) { @Override public void visit(@NotNull final AggSpecAbsSum absSum) { - visitBasicAgg(IterativeOperatorSpec::getAbsSumChunked); + addBasicOperators(IterativeOperatorSpec::getAbsSumChunked); } @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - visitBasicAgg((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), true, false)); + addBasicOperators((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), true, false)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - visitBasicAgg((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), true, false)); + addBasicOperators((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), true, false)); } @Override public void visit(@NotNull final AggSpecAvg avg) { - visitBasicAgg((t, n) -> getAvgChunked(t, n, true)); + addBasicOperators((t, n) -> getAvgChunked(t, n, true)); } @Override public void visit(@NotNull final AggSpecFirst first) { - // Supported + addFirstOrLastOperators(true, makeRedirectionName(nextColumnIdentifier++)); } @Override public void visit(@NotNull final AggSpecLast last) { - // Supported + addFirstOrLastOperators(false, makeRedirectionName(nextColumnIdentifier++)); } @Override public void visit(@NotNull final AggSpecMax max) { - visitMinOrMaxAgg(false); + addMinOrMaxOperators(false); } @Override public void visit(@NotNull final AggSpecMin min) { - visitMinOrMaxAgg(true); + addMinOrMaxOperators(true); } @Override public void visit(@NotNull final AggSpecSortedFirst sortedFirst) { - // Supported + addSortedFirstOrLastOperator(sortedFirst.columns(), true); } @Override public void visit(@NotNull final AggSpecSortedLast sortedLast) { - // Supported + addSortedFirstOrLastOperator(sortedLast.columns(), false); } @Override public void visit(@NotNull final AggSpecStd std) { - visitBasicAgg((t, n) -> getVarChunked(t, n, true, true)); + addBasicOperators((t, n) -> getVarChunked(t, n, true, true)); } @Override public void visit(@NotNull final AggSpecSum sum) { - visitBasicAgg(IterativeOperatorSpec::getSumChunked); + addBasicOperators(IterativeOperatorSpec::getSumChunked); } @Override public void visit(@NotNull final AggSpecUnique unique) { - visitBasicAgg((t, n) -> getUniqueChunked(t, n, + addBasicOperators((t, n) -> getUniqueChunked(t, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), true, false)); } @@ -625,7 +640,7 @@ public void visit(@NotNull final AggSpecWSum wSum) { @Override public void visit(@NotNull final AggSpecVar var) { - visitBasicAgg((t, n) -> getVarChunked(t, n, false, true)); + addBasicOperators((t, n) -> getVarChunked(t, n, false, true)); } } @@ -637,10 +652,12 @@ public void visit(@NotNull final AggSpecVar var) { * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an * {@link AggregationContext} for rollup reaggregated (not base level) aggregations. */ - private class RollupReaggregatedConverter extends Converter + private final class RollupReaggregatedConverter extends Converter implements RollupAggregation.Visitor, UnsupportedRollupAggregations { - protected RollupReaggregatedConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + private int nextColumnIdentifier = 0; + + private RollupReaggregatedConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { super(table, groupByColumnNames); } @@ -649,9 +666,8 @@ protected RollupReaggregatedConverter(@NotNull final Table table, @NotNull final // ------------------------------------------------------------------------------------------------------------- @Override - public final void visit(@NotNull final Count count) { - // TODO-RWC: Re-agg as sum - addNoInputOperator(new CountAggregationOperator(count.column().name())); + public void visit(@NotNull final Count count) { + addNoInputOperator(new LongChunkedSumOperator(false, count.column().name())); } @Override @@ -684,77 +700,247 @@ public void visit(@NotNull final Partition partition) { @Override public void visit(@NotNull final AggSpecAbsSum absSum) { - + reaggregateAsSum(); } @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - + reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> getCountDistinctChunked(ssmType, n, + countDistinct.countNulls(), true, true)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - + reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> getDistinctChunked(priorResultType, n, + distinct.includeNulls(), true, true)); } @Override public void visit(@NotNull final AggSpecAvg avg) { - + reaggregateAvgOperator(); } @Override public void visit(@NotNull final AggSpecFirst first) { - + reaggregateFirstOrLastOperator(true); } @Override public void visit(@NotNull final AggSpecLast last) { - + reaggregateFirstOrLastOperator(false); } @Override public void visit(@NotNull final AggSpecMax max) { - + reaggregateMinOrMaxOperators(false); } @Override public void visit(@NotNull final AggSpecMin min) { - + reaggregateMinOrMaxOperators(true); } @Override public void visit(@NotNull final AggSpecSortedFirst sortedFirst) { - + reaggregateSortedFirstOrLastOperator(sortedFirst.columns(), true); } @Override public void visit(@NotNull final AggSpecSortedLast sortedLast) { - + reaggregateSortedFirstOrLastOperator(sortedLast.columns(), false); } @Override public void visit(@NotNull final AggSpecStd std) { - + reaggregateStdOrVarOperators(true); } @Override public void visit(@NotNull final AggSpecSum sum) { - + reaggregateAsSum(); } @Override public void visit(@NotNull final AggSpecUnique unique) { - + reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> getUniqueChunked( + priorResultType, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), true, true)); } @Override public void visit(@NotNull final AggSpecWSum wSum) { - // Re-agg as sum + reaggregateAsSum(); } @Override public void visit(@NotNull final AggSpecVar var) { + reaggregateStdOrVarOperators(false); + } + + private void reaggregateAsSum() { + for (final Pair pair : resultPairs) { + final String resultName = pair.output().name(); + final ColumnSource resultSource = table.getColumnSource(resultName); + + addOperator(IterativeOperatorSpec.getSumChunked(resultSource.getType(), resultName), + resultSource, resultName); + } + } + + private void reaggregateSsmBackedOperator( + TriFunction, Class, String, IterativeChunkedAggregationOperator> operatorFactory) { + for (final Pair pair : resultPairs) { + final String resultName = pair.output().name(); + final String ssmName = resultName + ROLLUP_DISTINCT_SSM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final ColumnSource> ssmSource = table.getColumnSource(ssmName); + final ColumnSource priorResultSource = table.getColumnSource(resultName); + final IterativeChunkedAggregationOperator operator = operatorFactory.apply( + ssmSource.getComponentType(), priorResultSource.getComponentType(), resultName); + + addOperator(operator, ssmSource, ssmName); + } + } + + private void reaggregateFirstOrLastOperator(final boolean isFirst) { + final ColumnName redirectionColumnName = ColumnName.of(makeRedirectionName(nextColumnIdentifier++)); + resultPairs = Stream.concat( + resultPairs.stream().map(Pair::output), + Stream.of(redirectionColumnName)) + .collect(Collectors.toList()); + addSortedFirstOrLastOperator(List.of(SortColumn.asc(redirectionColumnName)), isFirst); + } + + private void reaggregateSortedFirstOrLastOperator( + @NotNull final List sortColumns, final boolean isFirst) { + resultPairs = resultPairs.stream().map(Pair::output).collect(Collectors.toList()); + addSortedFirstOrLastOperator(sortColumns, isFirst); + } + + private void reaggregateMinOrMaxOperators(final boolean isMin) { + for (final Pair pair : resultPairs) { + final String resultName = pair.output().name(); + addMinOrMaxOperator(isMin, resultName, resultName); + } + } + + private void reaggregateAvgOperator() { + for (final Pair pair : resultPairs) { + final String resultName = pair.output().name(); + + final String runningSumName = resultName + ROLLUP_RUNNING_SUM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final Class runningSumType = table.getColumnSource(runningSumName).getType(); + + final String nonNullCountName = resultName + ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator nonNullCountOp = addAndGetLongSumOperator(nonNullCountName); + + if (runningSumType == double.class) { + final DoubleChunkedSumOperator runningSumOp = addAndGetDoubleSumOperator(runningSumName); + + final String nanCountName = resultName + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator nanCountOp = addAndGetLongSumOperator(nanCountName); + + final String piCountName = resultName + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator piCountOp = addAndGetLongSumOperator(piCountName); + + final String niCountName = resultName + ROLLUP_NI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator niCountOp = addAndGetLongSumOperator(niCountName); + + final Class resultType = table.getColumnSource(resultName).getType(); + if (resultType == float.class) { + addOperator(new FloatChunkedReAvgOperator(resultName, + runningSumOp, nonNullCountOp, nanCountOp, piCountOp, niCountOp), + null, nonNullCountName, runningSumName, nanCountName, piCountName, niCountName); + } else { // resultType == double.class + addOperator(new DoubleChunkedReAvgOperator(resultName, + runningSumOp, nonNullCountOp, nanCountOp, piCountOp, niCountOp), + null, nonNullCountName, runningSumName, nanCountName, piCountName, niCountName); + } + } else if (BigInteger.class.isAssignableFrom(runningSumType)) { + final BigIntegerChunkedSumOperator runningSumOp = addAndGetBigIntegerSumOperator(runningSumName); + addOperator(new BigIntegerChunkedReAvgOperator(resultName, runningSumOp, nonNullCountOp), + null, nonNullCountName, runningSumName); + } else if (BigDecimal.class.isAssignableFrom(runningSumType)) { + final BigDecimalChunkedSumOperator runningSumOp = addAndGetBigDecimalSumOperator(runningSumName); + addOperator(new BigDecimalChunkedReAvgOperator(resultName, runningSumOp, nonNullCountOp), + null, nonNullCountName, runningSumName); + } else { + final LongChunkedSumOperator runningSumOp = addAndGetLongSumOperator(runningSumName); + addOperator(new IntegralChunkedReAvgOperator(resultName, runningSumOp, nonNullCountOp), + null, nonNullCountName, runningSumName); + } + } + } + + private void reaggregateStdOrVarOperators(final boolean isStd) { + for (final Pair pair : resultPairs) { + final String resultName = pair.output().name(); + + final String runningSumName = resultName + ROLLUP_RUNNING_SUM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final Class runningSumType = table.getColumnSource(runningSumName).getType(); + + final String runningSum2Name = resultName + ROLLUP_RUNNING_SUM2_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + + final String nonNullCountName = resultName + ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator nonNullCountOp = addAndGetLongSumOperator(nonNullCountName); + + if (runningSumType == double.class) { + final DoubleChunkedSumOperator runningSumOp = addAndGetDoubleSumOperator(runningSumName); + final DoubleChunkedSumOperator runningSum2Op = addAndGetDoubleSumOperator(runningSum2Name); + + final String nanCountName = resultName + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator nanCountOp = addAndGetLongSumOperator(nanCountName); + + final String piCountName = resultName + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator piCountOp = addAndGetLongSumOperator(piCountName); + + final String niCountName = resultName + ROLLUP_NI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + final LongChunkedSumOperator niCountOp = addAndGetLongSumOperator(niCountName); + + addOperator(new FloatChunkedReVarOperator(resultName, isStd, runningSumOp, runningSum2Op, + nonNullCountOp, nanCountOp, piCountOp, niCountOp), null, + nonNullCountName, runningSumName, runningSum2Name, nanCountName, piCountName, niCountName); + } else if (BigInteger.class.isAssignableFrom(runningSumType)) { + final BigIntegerChunkedSumOperator runningSumOp = addAndGetBigIntegerSumOperator(runningSumName); + final BigIntegerChunkedSumOperator runningSum2Op = addAndGetBigIntegerSumOperator(runningSum2Name); + addOperator(new BigIntegerChunkedReVarOperator(resultName, isStd, + runningSumOp, runningSum2Op, nonNullCountOp), + null, nonNullCountName, runningSumName, runningSum2Name); + } else if (BigDecimal.class.isAssignableFrom(runningSumType)) { + final BigDecimalChunkedSumOperator runningSumOp = addAndGetBigDecimalSumOperator(runningSumName); + final BigDecimalChunkedSumOperator runningSum2Op = addAndGetBigDecimalSumOperator(runningSum2Name); + addOperator(new BigDecimalChunkedReVarOperator(resultName, isStd, + runningSumOp, runningSum2Op, nonNullCountOp), + null, nonNullCountName, runningSumName, runningSum2Name); + } else { + final DoubleChunkedSumOperator runningSumOp = addAndGetDoubleSumOperator(runningSumName); + final DoubleChunkedSumOperator runningSum2Op = addAndGetDoubleSumOperator(runningSum2Name); + addOperator(new IntegralChunkedReVarOperator(resultName, isStd, + runningSumOp, runningSum2Op, nonNullCountOp), + null, nonNullCountName, runningSumName, runningSum2Name); + } + } + } + + private BigDecimalChunkedSumOperator addAndGetBigDecimalSumOperator(@NotNull final String inputColumnName) { + return getAndAddBasicOperator(n -> new BigDecimalChunkedSumOperator(false, n), inputColumnName); + } + + private BigIntegerChunkedSumOperator addAndGetBigIntegerSumOperator(@NotNull final String inputColumnName) { + return getAndAddBasicOperator(n -> new BigIntegerChunkedSumOperator(false, n), inputColumnName); + } + + private DoubleChunkedSumOperator addAndGetDoubleSumOperator(@NotNull final String inputColumnName) { + return getAndAddBasicOperator(n -> new DoubleChunkedSumOperator(false, n), inputColumnName); + } + + private LongChunkedSumOperator addAndGetLongSumOperator(@NotNull final String inputColumnName) { + return getAndAddBasicOperator(n -> new LongChunkedSumOperator(false, n), inputColumnName); + } + private OP_TYPE getAndAddBasicOperator( + @NotNull final Function opFactory, @NotNull final String inputColumnName) { + OP_TYPE operator = opFactory.apply(inputColumnName); + addOperator(operator, table.getColumnSource(inputColumnName), inputColumnName); + return operator; } } @@ -768,10 +954,8 @@ private static ColumnSource maybeReinterpretDateTimeAsLong(@NotNull final Col : inputSource; } - @NotNull - private static String makeRedirectionName(IterativeIndexSpec inputAggregationStateFactory) { - return IterativeIndexSpec.ROW_REDIRECTION_PREFIX + inputAggregationStateFactory.rollupColumnIdentifier - + RollupConstants.ROLLUP_COLUMN_SUFFIX; + private static String makeRedirectionName(final int columnIdentifier) { + return ROW_REDIRECTION_PREFIX + columnIdentifier + ROLLUP_COLUMN_SUFFIX; } private static QueryTable maybeCopyRlAttribute(@NotNull final Table parent, @NotNull final Table child) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java index efa3f7c927c..b6b58ca00a0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java @@ -203,20 +203,20 @@ void startTrackingPrevFpCounterValues() { Map> fpInternalColumnSources(final String name) { final Map> results = new LinkedHashMap<>(); if (nanCount != null) { - results.put(name + ROLLUP_NAN_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, nanCount); + results.put(name + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, nanCount); } else { - results.put(name + ROLLUP_NAN_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, new WrappedLongArraySource(() -> nanCount)); + results.put(name + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, new WrappedLongArraySource(() -> nanCount)); } if (positiveInfinityCount != null) { - results.put(name + ROLLUP_PIC_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, positiveInfinityCount); + results.put(name + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, positiveInfinityCount); } else { - results.put(name + ROLLUP_PIC_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, + results.put(name + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, new WrappedLongArraySource(() -> positiveInfinityCount)); } if (negativeInfinityCount != null) { - results.put(name + ROLLUP_NIC_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, negativeInfinityCount); + results.put(name + ROLLUP_NI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, negativeInfinityCount); } else { - results.put(name + ROLLUP_NIC_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, + results.put(name + ROLLUP_NI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, new WrappedLongArraySource(() -> negativeInfinityCount)); } return results; From 2849e65ff532fa892baac1b0ecdaab80f069e769 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 26 Jan 2022 00:27:26 -0500 Subject: [PATCH 15/44] WIP on integration --- .../table/impl/MemoizedOperationKey.java | 81 ++++----- .../engine/table/impl/QueryTable.java | 157 ++++-------------- .../table/impl/by/AggregationProcessor.java | 9 +- .../table/impl/by/rollup/NullColumns.java | 8 + 4 files changed, 79 insertions(+), 176 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java index 1461195b4b2..f3a29f242a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java @@ -1,19 +1,15 @@ package io.deephaven.engine.table.impl; +import io.deephaven.api.agg.Aggregation; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.WouldMatchPair; -import io.deephaven.engine.table.impl.by.AggregationFactory; -import io.deephaven.engine.table.impl.by.AggregationMemoKey; -import io.deephaven.engine.table.impl.by.AggregationSpec; import io.deephaven.engine.table.impl.select.*; import io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource; import org.jetbrains.annotations.NotNull; import java.lang.ref.WeakReference; -import java.util.Arrays; -import java.util.Map; -import java.util.Objects; +import java.util.*; /** * Indices for memoized operations on QueryTable. @@ -99,16 +95,12 @@ public static MemoizedOperationKey treeTable(String idColumn, String parentColum return new TreeTable(idColumn, parentColumn); } - public static MemoizedOperationKey by(AggregationSpec aggregationSpec, - SelectColumn[] groupByColumns) { + public static MemoizedOperationKey aggBy(Collection aggregations, + SelectColumn[] groupByColumns) { if (!isMemoizable(groupByColumns)) { return null; } - final AggregationMemoKey aggregationMemoKey = aggregationSpec.getMemoKey(); - if (aggregationMemoKey == null) { - return null; - } - return new GroupBy(aggregationMemoKey, groupByColumns); + return new AggBy(aggregations.toArray(new Aggregation[0]), groupByColumns); } public static MemoizedOperationKey partitionBy(boolean dropKeys, SelectColumn[] groupByColumns) { @@ -118,22 +110,17 @@ public static MemoizedOperationKey partitionBy(boolean dropKeys, SelectColumn[] return new PartitionBy(dropKeys, groupByColumns); } - private static boolean isMemoizable(SelectColumn[] selectColumn) { - return Arrays.stream(selectColumn) - .allMatch(sc -> sc instanceof SourceColumn || sc instanceof ReinterpretedColumn); - } - - public static MemoizedOperationKey rollup(AggregationFactory aggregationFactory, SelectColumn[] columns, - boolean includeConstituents) { - if (!isMemoizable(columns)) { + public static MemoizedOperationKey rollup(Collection aggregations, + SelectColumn[] groupByColumns, boolean includeConstituents) { + if (!isMemoizable(groupByColumns)) { return null; } + return new Rollup(aggregations.toArray(new Aggregation[0]), groupByColumns, includeConstituents); + } - final AggregationMemoKey aggregationMemoKey = aggregationFactory.getMemoKey(); - if (aggregationMemoKey == null) { - return null; - } - return new Rollup(aggregationMemoKey, columns, includeConstituents); + private static boolean isMemoizable(SelectColumn[] selectColumn) { + return Arrays.stream(selectColumn) + .allMatch(sc -> sc instanceof SourceColumn || sc instanceof ReinterpretedColumn); } private static class Flatten extends AttributeAgnosticMemoizedOperationKey { @@ -349,12 +336,13 @@ public int hashCode() { } } - private static class GroupBy extends AttributeAgnosticMemoizedOperationKey { - private final AggregationMemoKey aggregationKey; + private static class AggBy extends AttributeAgnosticMemoizedOperationKey { + + private final Aggregation[] aggregations; private final SelectColumn[] groupByColumns; - private GroupBy(@NotNull AggregationMemoKey aggregationKey, SelectColumn[] groupByColumns) { - this.aggregationKey = aggregationKey; + private AggBy(Aggregation[] aggregations, SelectColumn[] groupByColumns) { + this.aggregations = aggregations; this.groupByColumns = groupByColumns; } @@ -364,19 +352,18 @@ public boolean equals(Object o) { return true; if (o == null || getClass() != o.getClass()) return false; - final GroupBy groupBy = (GroupBy) o; - return Objects.equals(aggregationKey, groupBy.aggregationKey) && - Arrays.equals(groupByColumns, groupBy.groupByColumns); + final AggBy aggBy = (AggBy) o; + return Arrays.equals(aggregations, aggBy.aggregations) && + Arrays.equals(groupByColumns, aggBy.groupByColumns); } @Override public int hashCode() { - int result = Objects.hash(aggregationKey); + int result = Arrays.hashCode(aggregations); result = 31 * result + Arrays.hashCode(groupByColumns); return result; } - @Override BaseTable.CopyAttributeOperation copyType() { return BaseTable.CopyAttributeOperation.None; @@ -384,6 +371,7 @@ BaseTable.CopyAttributeOperation copyType() { } static class PartitionBy extends MemoizedOperationKey { + private final boolean dropKeys; private final SelectColumn[] groupByColumns; @@ -398,31 +386,32 @@ public boolean equals(Object o) { return true; if (o == null || getClass() != o.getClass()) return false; - final PartitionBy by = (PartitionBy) o; - return dropKeys == by.dropKeys && - Arrays.equals(groupByColumns, by.groupByColumns); + final PartitionBy partitionBy = (PartitionBy) o; + return dropKeys == partitionBy.dropKeys && + Arrays.equals(groupByColumns, partitionBy.groupByColumns); } @Override public int hashCode() { - int result = Objects.hash(dropKeys); + int result = Boolean.hashCode(dropKeys); result = 31 * result + Arrays.hashCode(groupByColumns); return result; } } private static class Rollup extends AttributeAgnosticMemoizedOperationKey { - private final GroupBy groupBy; + + private final AggBy aggBy; private final boolean includeConstituents; - Rollup(@NotNull AggregationMemoKey aggregationKey, SelectColumn[] groupByColumns, boolean includeConstituents) { + Rollup(Aggregation[] aggregations, SelectColumn[] groupByColumns, boolean includeConstituents) { this.includeConstituents = includeConstituents; - this.groupBy = new GroupBy(aggregationKey, groupByColumns); + this.aggBy = new AggBy(aggregations, groupByColumns); } @Override public int hashCode() { - return 31 * groupBy.hashCode() + Boolean.hashCode(includeConstituents); + return 31 * aggBy.hashCode() + Boolean.hashCode(includeConstituents); } @Override @@ -432,7 +421,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; final Rollup rollup = (Rollup) o; - return Objects.equals(groupBy, rollup.groupBy) && includeConstituents == rollup.includeConstituents; + return Objects.equals(aggBy, rollup.aggBy) && includeConstituents == rollup.includeConstituents; } @Override @@ -462,7 +451,7 @@ private SymbolTable(@NotNull final SymbolTableSource symbolTableSource, final bo } @Override - public final boolean equals(final Object other) { + public boolean equals(final Object other) { if (this == other) { return true; } @@ -475,7 +464,7 @@ public final boolean equals(final Object other) { } @Override - public final int hashCode() { + public int hashCode() { return 31 * System.identityHashCode(symbolTableSource) + Boolean.hashCode(useLookupCaching); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index 1cc8775056b..c681687d94f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -25,6 +25,8 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; +import io.deephaven.engine.table.impl.by.rollup.NullColumns; +import io.deephaven.engine.table.impl.by.rollup.Partition; import io.deephaven.engine.table.impl.indexer.RowSetIndexer; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; @@ -81,6 +83,8 @@ import java.util.stream.Stream; import static io.deephaven.engine.table.MatchPair.matchString; +import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; +import static io.deephaven.engine.table.impl.by.AggregationProcessor.Type.ROLLUP_BASE; /** * Primary coalesced table implementation. @@ -466,35 +470,28 @@ public LocalTableMap partitionBy(final boolean dropKeys, final String... keyColu @Override public Table rollup(Collection aggregations, boolean includeConstituents, Selectable... groupByColumns) { - final List converted = - AggregationFactory.AggregationElement.convert(aggregations); - return rollup(new AggregationFactory(converted), includeConstituents, SelectColumn.from(groupByColumns)); - } - - // TODO (https://github.com/deephaven/deephaven-core/issues/991): Make this private, and clean up everything that - // uses the AggregationFactory as a specifier. - public Table rollup(AggregationFactory aggregationFactory, boolean includeConstituents, SelectColumn... columns) { if (isStream() && includeConstituents) { throw streamUnsupported("rollup with included constituents"); } - return memoizeResult(MemoizedOperationKey.rollup(aggregationFactory, columns, includeConstituents), () -> { - final AggregationFactory withRollup = aggregationFactory.forRollup(includeConstituents); - AggregationFactory aggregationStateFactory = withRollup; - - final QueryTable lowestLevel = byNoMemo(withRollup, columns); - // now we need to reaggregate at each of the levels, combining the results - final List reaggregateColumns = new ArrayList<>(Arrays.asList(columns)); - - final AggregationFactory rollupFactory = withRollup.rollupFactory(); - - final List nullColumns = new ArrayList<>(reaggregateColumns.size()); - - QueryTable lastLevel = lowestLevel; + final SelectColumn[] gbsColumns = SelectColumn.from(groupByColumns); + final MemoizedOperationKey rollupKey = + MemoizedOperationKey.rollup(aggregations, gbsColumns, includeConstituents); + return memoizeResult(rollupKey, () -> { + final List baseAggregations = Stream.concat( + aggregations.stream(), + Stream.of(includeConstituents + ? Partition.of(true) + : NullColumns.of(ROLLUP_COLUMN, Object.class)) + ).collect(Collectors.toList()); + final QueryTable baseLevel = aggNoMemo(AggregationProcessor.of(baseAggregations, ROLLUP_BASE), gbsColumns); + + final Deque reaggregateColumns = new ArrayDeque<>(Arrays.asList(gbsColumns)); + final Deque nullColumns = new ArrayDeque<>(groupByColumns.length); + + QueryTable lastLevel = baseLevel; while (!reaggregateColumns.isEmpty()) { - final SelectColumn removedColumn = reaggregateColumns.remove(reaggregateColumns.size() - 1); - - nullColumns.add(0, removedColumn.getName()); - + nullColumns.addFirst(reaggregateColumns.removeLast().getName()); + // TODO-RWC: RESUME FROM HERE: FIGURE OUT WHEN TO ADD PARTITION final Map> nullColumnsMap = new LinkedHashMap<>(nullColumns.size()); final Table fLastLevel = lastLevel; nullColumns @@ -674,112 +671,16 @@ public Table countBy(String countColumnName, Selectable... groupByColumns) { // TODO (https://github.com/deephaven/deephaven-core/issues/991): Make this private, and clean up everything that // uses the AggregationFactory as a specifier. public Table by(final AggregationSpec inputAggregationSpec, final SelectColumn... groupByColumns) { - return memoizeResult(MemoizedOperationKey.by(inputAggregationSpec, groupByColumns), + return memoizeResult(MemoizedOperationKey.aggBy(inputAggregationSpec, groupByColumns), () -> byNoMemo(inputAggregationSpec, groupByColumns)); } - private QueryTable byNoMemo(AggregationSpec inputAggregationSpec, - final SelectColumn... groupByColumns) { - final String description = "by(" + inputAggregationSpec + ", " + Arrays.toString(groupByColumns) + ")"; - - return QueryPerformanceRecorder.withNugget(description, sizeForInstrumentation(), () -> { - - final boolean isBy = inputAggregationSpec.getClass() == AggregationGroupSpec.class; - final boolean isApplyToAllBy = - inputAggregationSpec.getClass() == AggregationFormulaSpec.class; - final boolean isNumeric = inputAggregationSpec.getClass() == SumSpec.class || - inputAggregationSpec.getClass() == AbsSumSpec.class || - inputAggregationSpec.getClass() == AvgSpec.class || - inputAggregationSpec.getClass() == VarSpec.class || - inputAggregationSpec.getClass() == StdSpec.class; - final boolean isSelectDistinct = - inputAggregationSpec.getClass() == SelectDistinctSpecImpl.class; - final boolean isCount = inputAggregationSpec.getClass() == CountBySpecImpl.class; - final boolean isMinMax = inputAggregationSpec instanceof MinMaxBySpecImpl; - final boolean isPercentile = inputAggregationSpec.getClass() == PercentileBySpecImpl.class; - final boolean isWeightedAvg = - inputAggregationSpec.getClass() == WeightedAverageSpecImpl.class; - final boolean isWeightedSum = inputAggregationSpec.getClass() == WeightedSumSpecImpl.class; - final boolean isSortedFirstOrLast = inputAggregationSpec instanceof SortedFirstOrLastByFactoryImpl; - final boolean isFirst = inputAggregationSpec.getClass() == FirstBySpecImpl.class - || inputAggregationSpec.getClass() == TrackingFirstBySpecImpl.class; - final boolean isLast = inputAggregationSpec.getClass() == LastBySpecImpl.class - || inputAggregationSpec.getClass() == TrackingLastBySpecImpl.class; - final boolean isCombo = inputAggregationSpec instanceof AggregationFactory; - - if (isBy) { - if (isStream()) { - throw streamUnsupported("groupBy"); - } - if (USE_OLDER_CHUNKED_BY) { - return AggregationHelper.by(this, groupByColumns); - } - return GroupByAggregationFactory.by(this, groupByColumns); - } else if (isApplyToAllBy) { - if (isStream()) { - throw streamUnsupported("applyToAllBy"); - } - final String formula = ((AggregationFormulaSpec) inputAggregationSpec).getFormula(); - final String columnParamName = - ((AggregationFormulaSpec) inputAggregationSpec).getColumnParamName(); - return FormulaAggregationFactory.applyToAllBy(this, formula, columnParamName, groupByColumns); - } else if (isNumeric) { - return ChunkedOperatorAggregationHelper.aggregation(new NonKeyColumnAggregationFactory( - (IterativeChunkedOperatorFactory) inputAggregationSpec), this, groupByColumns); - } else if (isSortedFirstOrLast) { - final boolean isSortedFirst = - ((SortedFirstOrLastByFactoryImpl) inputAggregationSpec).isSortedFirst(); - return ChunkedOperatorAggregationHelper.aggregation( - new SortedFirstOrLastByAggregationFactory(isSortedFirst, false, - ((SortedFirstOrLastByFactoryImpl) inputAggregationSpec).getSortColumnNames()), - this, groupByColumns); - } else if (isFirst || isLast) { - return ChunkedOperatorAggregationHelper.aggregation(new FirstOrLastByAggregationFactory(isFirst), this, - groupByColumns); - } else if (isMinMax) { - final boolean isMin = ((MinMaxBySpecImpl) inputAggregationSpec).isMinimum(); - return ChunkedOperatorAggregationHelper.aggregation( - new NonKeyColumnAggregationFactory( - new MinMaxIterativeOperatorFactory(isMin, isStream() || isAddOnly())), - this, groupByColumns); - } else if (isPercentile) { - final double percentile = ((PercentileBySpecImpl) inputAggregationSpec).getPercentile(); - final boolean averageMedian = - ((PercentileBySpecImpl) inputAggregationSpec).getAverageMedian(); - return ChunkedOperatorAggregationHelper.aggregation( - new NonKeyColumnAggregationFactory( - new PercentileIterativeOperatorFactory(percentile, averageMedian)), - this, groupByColumns); - } else if (isWeightedAvg || isWeightedSum) { - final String weightName; - if (isWeightedAvg) { - weightName = ((WeightedAverageSpecImpl) inputAggregationSpec).getWeightName(); - } else { - weightName = ((WeightedSumSpecImpl) inputAggregationSpec).getWeightName(); - } - return ChunkedOperatorAggregationHelper.aggregation( - new WeightedAverageSumAggregationFactory(weightName, isWeightedSum), this, groupByColumns); - } else if (isCount) { - return ChunkedOperatorAggregationHelper.aggregation( - new CountAggregationFactory( - ((CountBySpecImpl) inputAggregationSpec).getCountName()), - this, groupByColumns); - } else if (isSelectDistinct) { - if (getColumnSourceMap().isEmpty()) { - // if we have no input columns, then the only thing we can do is have an empty result - return new QueryTable(RowSetFactory.empty().toTracking(), - Collections.emptyMap()); - } - return ChunkedOperatorAggregationHelper.aggregation(new KeyOnlyAggregationFactory(), this, - groupByColumns); - } else if (isCombo) { - return ChunkedOperatorAggregationHelper.aggregation( - ((AggregationFactory) inputAggregationSpec).makeAggregationContextFactory(), this, - groupByColumns); - } - - throw new RuntimeException("Unknown aggregation : " + inputAggregationSpec); - }); + private QueryTable aggNoMemo(@NotNull final AggregationContextFactory aggregationContextFactory, + @NotNull final SelectColumn... groupByColumns) { + final String description = "aggregation(" + aggregationContextFactory + + ", " + Arrays.toString(groupByColumns) + ")"; + return QueryPerformanceRecorder.withNugget(description, sizeForInstrumentation(), + () -> ChunkedOperatorAggregationHelper.aggregation(aggregationContextFactory, this, groupByColumns)); } private static UnsupportedOperationException streamUnsupported(@NotNull final String operationName) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index fdf657d75b3..2f96077741a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -55,7 +55,7 @@ public static AggregationContextFactory of(@NotNull final Collection aggregations; @@ -75,6 +75,11 @@ private AggregationProcessor( } } + @Override + public String toString() { + return type.name() + ':' + aggregations; + } + // ----------------------------------------------------------------------------------------------------------------- // AggregationContextFactory // ----------------------------------------------------------------------------------------------------------------- @@ -83,7 +88,7 @@ private AggregationProcessor( public AggregationContext makeAggregationContext(@NotNull final Table table, @NotNull final String... groupByColumnNames) { switch (type) { - case STANDARD: + case NORMAL: return new StandardConverter(table, groupByColumnNames).build(); case ROLLUP_BASE: return new RollupBaseConverter(table, groupByColumnNames).build(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java index 88577564e4d..dc6122ce984 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java @@ -12,6 +12,14 @@ @BuildableStyle public abstract class NullColumns extends RollupAggregationBase { + public static NullColumns of(String name, Class type) { + return builder().putResultColumns(name, type).build(); + } + + public static NullColumns from(Map> resultColumns) { + return builder().putResultColumns(resultColumns).build(); + } + public static Builder builder() { return ImmutableNullColumns.builder(); } From d3f233190efb27e4ae8d60bcc3cf0a3d155505b7 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 26 Jan 2022 23:56:59 -0500 Subject: [PATCH 16/44] Tons of progress on gluing it all together. Description support for aggs. --- .../deephaven/treetable/RollupDefinition.java | 323 ++------- .../main/java/io/deephaven/plot/AxesImpl.java | 25 +- .../io/deephaven/plot/util/PlotUtils.java | 19 +- .../engine/PercentileByBenchmark.java | 4 +- .../engine/table/impl/QueryTable.java | 104 ++- .../engine/table/impl/RollupInfo.java | 17 +- .../table/impl/by/AggregationProcessor.java | 655 ++++++++++++++++-- .../api/agg/AggregationDescriptions.java | 57 ++ .../deephaven/api/agg/AggregationOutputs.java | 42 +- .../deephaven/api/agg/AggregationPairs.java | 60 ++ .../io/deephaven/api/agg/spec/AggSpec.java | 2 + .../deephaven/api/agg/spec/AggSpecAbsSum.java | 5 + .../io/deephaven/api/agg/spec/AggSpecAvg.java | 5 + .../api/agg/spec/AggSpecCountDistinct.java | 5 + .../api/agg/spec/AggSpecDistinct.java | 5 + .../deephaven/api/agg/spec/AggSpecFirst.java | 5 + .../api/agg/spec/AggSpecFormula.java | 7 + .../deephaven/api/agg/spec/AggSpecGroup.java | 5 + .../deephaven/api/agg/spec/AggSpecLast.java | 5 + .../io/deephaven/api/agg/spec/AggSpecMax.java | 5 + .../deephaven/api/agg/spec/AggSpecMedian.java | 5 + .../io/deephaven/api/agg/spec/AggSpecMin.java | 5 + .../api/agg/spec/AggSpecPercentile.java | 5 + .../api/agg/spec/AggSpecSortedFirst.java | 6 + .../api/agg/spec/AggSpecSortedLast.java | 6 + .../io/deephaven/api/agg/spec/AggSpecStd.java | 5 + .../io/deephaven/api/agg/spec/AggSpecSum.java | 5 + .../deephaven/api/agg/spec/AggSpecUnique.java | 5 + .../io/deephaven/api/agg/spec/AggSpecVar.java | 5 + .../deephaven/api/agg/spec/AggSpecWAvg.java | 5 + .../deephaven/api/agg/spec/AggSpecWSum.java | 5 + 31 files changed, 976 insertions(+), 436 deletions(-) create mode 100644 table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java diff --git a/ClientSupport/src/main/java/io/deephaven/treetable/RollupDefinition.java b/ClientSupport/src/main/java/io/deephaven/treetable/RollupDefinition.java index 7a2c9c954c4..f742ce309f5 100644 --- a/ClientSupport/src/main/java/io/deephaven/treetable/RollupDefinition.java +++ b/ClientSupport/src/main/java/io/deephaven/treetable/RollupDefinition.java @@ -1,38 +1,26 @@ package io.deephaven.treetable; +import io.deephaven.UncheckedDeephavenException; import io.deephaven.api.Selectable; -import io.deephaven.engine.table.Table; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.AggregationDescriptions; +import io.deephaven.api.agg.AggregationPairs; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.util.string.StringUtils; -import io.deephaven.engine.table.impl.by.AggType; -import io.deephaven.engine.table.impl.by.AggregationFactory; -import gnu.trove.map.hash.TObjectIntHashMap; -import io.deephaven.UncheckedDeephavenException; -import org.jdom2.Element; -import java.io.Serializable; import java.util.*; +import java.util.stream.Collectors; /** - * This class defines a rollup. It defines both the grouping columns, their order, and all aggregations to be performed + * This class defines a rollup. It defines both the group-by columns, their order, and all aggregations to be performed * on other columns. */ -public class RollupDefinition implements Serializable { - private static final long serialVersionUID = 2L; - - public static final String NODE_NAME = "RollupDefinition"; - private static final String GROUP_BY_NODE = "GroupBy"; - private static final String COLUMN_NODE = "Column"; - private static final String OPS_NODE = "Agg"; - private static final String ATTR_NAME = "name"; - private static final String ATTR_CONSTITUENTS = "constituents"; - private static final String ATTR_INCLUDE_OTHER = "includeOther"; - private static final String ATTR_INCLUDE_DESCRIPTIONS = "includeDescriptions"; - - private final List groupingColumns; - private final Map> aggregations; +public class RollupDefinition { + + private final List aggregations; + private final List groupByColumns; private final boolean includeConstituents; private final boolean includeOriginalColumns; private final boolean includeDescriptions; @@ -40,34 +28,32 @@ public class RollupDefinition implements Serializable { /** * Create a RollupDefinition. - * - * @param groupingColumns the columns to group by, order matters - * @param aggregations the aggregations to perform and which columns to perform them on + * + * @param aggregations the aggregations to perform + * @param groupByColumns the columns to group by, order matters * @param includeConstituents if constituent rows should be included * @param includeOriginalColumns if original columns should be included * @param includeDescriptions if the rollup should automatically add column descriptions for the chosen aggs */ - public RollupDefinition(List groupingColumns, Map> aggregations, + public RollupDefinition(List aggregations, List groupByColumns, boolean includeConstituents, boolean includeOriginalColumns, boolean includeDescriptions) { - this(groupingColumns, aggregations, includeConstituents, includeOriginalColumns, includeDescriptions, ""); + this(aggregations, groupByColumns, includeConstituents, includeOriginalColumns, includeDescriptions, ""); } /** * Create a RollupDefinition. - * - * @param groupingColumns the columns to group by, order matters - * @param aggregations the aggregations to perform and which columns to perform them on + * + * @param aggregations the aggregations to perform + * @param groupByColumns the columns to group by, order matters * @param includeConstituents if constituent rows should be included * @param includeOriginalColumns if original columns should be included * @param includeDescriptions if the rollup should automatically add column descriptions for the chosen aggs * @param name an optional name. */ - public RollupDefinition(List groupingColumns, Map> aggregations, + public RollupDefinition(List aggregations, List groupByColumns, boolean includeConstituents, boolean includeOriginalColumns, boolean includeDescriptions, String name) { - this.groupingColumns = new ArrayList<>(groupingColumns); - this.aggregations = new LinkedHashMap<>(); - aggregations.forEach( - (agg, cols) -> this.aggregations.computeIfAbsent(agg, a -> new LinkedHashSet<>()).addAll(cols)); + this.groupByColumns = new ArrayList<>(groupByColumns); + this.aggregations = new ArrayList<>(aggregations); this.includeConstituents = includeConstituents; this.includeOriginalColumns = includeOriginalColumns; this.includeDescriptions = includeDescriptions; @@ -80,32 +66,26 @@ public RollupDefinition(List groupingColumns, Map> * @param other the definition to copy */ public RollupDefinition(RollupDefinition other) { - this.groupingColumns = new ArrayList<>(other.groupingColumns); - this.aggregations = new LinkedHashMap<>(); - other.aggregations.forEach( - (agg, cols) -> this.aggregations.computeIfAbsent(agg, a -> new LinkedHashSet<>()).addAll(cols)); - this.includeConstituents = other.includeConstituents; - this.includeOriginalColumns = other.includeOriginalColumns; - this.includeDescriptions = other.includeDescriptions; - this.name = other.name; + this(other.aggregations, other.groupByColumns, + other.includeConstituents, other.includeOriginalColumns, other.includeDescriptions, other.name); } /** - * Get the grouping columns for this definition. + * Get the aggregations and applicable columns for this definition. * - * @return an unmodifiable list of grouping columns + * @return an unmodifiable map of aggregations to set of columns */ - public List getGroupingColumns() { - return Collections.unmodifiableList(groupingColumns); + public List getAggregations() { + return Collections.unmodifiableList(aggregations); } /** - * Get the aggregations and applicable columns for this definition. + * Get the group-by columns for this definition. * - * @return an unmodifiable map of aggregations to set of columns + * @return an unmodifiable list of group-by columns */ - public Map> getAggregations() { - return Collections.unmodifiableMap(aggregations); + public List getGroupByColumns() { + return Collections.unmodifiableList(groupByColumns); } /** @@ -136,43 +116,12 @@ public boolean includeDescriptions() { } /** - * Convert this definition into an XML {@link Element} suitable for use with workspaces. + * Get the resulting input/output pairs as {@link MatchPair match pairs}. * - * @return an XML representation of this definition + * @return The resulting match pairs */ - public Element toXml() { - final Element info = new Element(NODE_NAME) - .setAttribute(ATTR_CONSTITUENTS, Boolean.toString(includeConstituents)) - .setAttribute(ATTR_INCLUDE_OTHER, Boolean.toString(includeOriginalColumns)) - .setAttribute(ATTR_INCLUDE_DESCRIPTIONS, Boolean.toString(includeDescriptions)); - - if (!StringUtils.isNullOrEmpty(name)) { - info.setAttribute(ATTR_NAME, name); - } - - for (final String name : groupingColumns) { - final Element groupByElem = new Element(GROUP_BY_NODE); - groupByElem.setAttribute(ATTR_NAME, name); - info.addContent(groupByElem); - } - - for (final Map.Entry> item : aggregations.entrySet()) { - final Element opElem = new Element(OPS_NODE); - opElem.setAttribute(ATTR_NAME, item.getKey().toString()); - - item.getValue().stream() - .map(col -> new Element(COLUMN_NODE).setAttribute(ATTR_NAME, col)) - .forEach(opElem::addContent); - - info.addContent(opElem); - } - - return info; - } - public List getResultMatchPairs() { - final AggregationFactory caf = createComboAggregateFactory(null); - return caf.getMatchPairs(); + return AggregationPairs.of(aggregations).map(MatchPair::of).collect(Collectors.toList()); } /** @@ -200,107 +149,17 @@ public String getName() { * @return a rollup, as defined by this object. */ public Table applyTo(Table table) { - final Map maybeDescriptions = includeDescriptions ? new HashMap<>() : null; - Table result = ((QueryTable) table).rollup(createComboAggregateFactory(maybeDescriptions), includeConstituents, - SelectColumn.from(Selectable.from(groupingColumns))); - if (maybeDescriptions != null) { - result = result.withColumnDescription(maybeDescriptions); + Table result = table.rollup(aggregations, includeConstituents, + SelectColumn.from(Selectable.from(groupByColumns))); + if (includeDescriptions) { + result = result.withColumnDescription(AggregationDescriptions.of(aggregations)); } - return result; } - /** - * Create the AggregationFactory for this rollup. Generate column descriptions if required. - * - * @param descriptions if non-null this method will generate column descriptions - * @return the ComboAggFactory - */ - private AggregationFactory createComboAggregateFactory(final Map descriptions) { - final TObjectIntHashMap aggsByColumn = new TObjectIntHashMap<>(); - final List combos = new ArrayList<>(getAggregations().size()); - - // Take two passes through the list. The first pass is to decide if we need to append suffixes. - // The second pass actually creates the aggs. - for (final Map.Entry> item : getAggregations().entrySet()) { - if (item.getKey() != AggType.Count) { - item.getValue().forEach(c -> aggsByColumn.adjustOrPutValue(c, 1, 1)); - } - } - - for (final Map.Entry> item : getAggregations().entrySet()) { - if (item.getKey() == AggType.Count) { - combos.add(AggregationFactory.AggCount(item.getValue().stream().findFirst().orElse("Rollup_Count"))); - } else { - final String[] matchPairs = item.getValue() - .stream() - .map(col -> { - final String aggColName = createAggColName(col, aggsByColumn, item.getKey()); - if (descriptions != null) { - descriptions.put(aggColName, col + " aggregated with " + item.getKey()); - } - - return aggColName + "=" + col; - }).toArray(String[]::new); - combos.add(AggregationFactory.Agg(item.getKey(), matchPairs)); - } - } - - return AggregationFactory.AggCombo(combos.toArray(new AggregationFactory.AggregationElement[0])); - } - - private String createAggColName(String col, TObjectIntHashMap aggsByColumn, AggType agg) { - return (aggsByColumn.get(col) > 1 && agg != AggType.Sum) || groupingColumns.contains(col) ? col + "_" + agg - : col; - } - - /** - * Create a RollupDefinition from the specified XML element created by {@link #toXml()}. - * - * @param rollupElement the element - * @return a RollupDefinition - */ - public static RollupDefinition fromXml(Element rollupElement) throws UncheckedDeephavenException { - final List groupingColumns = new ArrayList<>(); - for (final Element groupByElem : rollupElement.getChildren(GROUP_BY_NODE)) { - groupingColumns.add(groupByElem.getAttributeValue(ATTR_NAME)); - } - - final Map> aggs = new LinkedHashMap<>(); - for (final Element opsElem : rollupElement.getChildren(OPS_NODE)) { - final String name = opsElem.getAttributeValue(ATTR_NAME); - if (StringUtils.isNullOrEmpty(name)) { - throw new UncheckedDeephavenException("Rollup element missing attribute name"); - } - - final AggType agg; - try { - agg = AggType.valueOf(name); - } catch (IllegalArgumentException e) { - throw new UncheckedDeephavenException("Unknown aggregation type:", e); - } - final Set colsForAgg = aggs.computeIfAbsent(agg, a -> new LinkedHashSet<>()); - - for (final Element colEl : opsElem.getChildren(COLUMN_NODE)) { - final String colName = colEl.getAttributeValue(ATTR_NAME); - if (StringUtils.isNullOrEmpty(colName)) { - throw new UncheckedDeephavenException("Rollup aggregation column element missing name"); - } - - colsForAgg.add(colName); - } - } - - return new RollupDefinition(groupingColumns, aggs, - Boolean.parseBoolean(rollupElement.getAttributeValue(ATTR_CONSTITUENTS)), - Boolean.parseBoolean(rollupElement.getAttributeValue(ATTR_INCLUDE_OTHER)), - Boolean.parseBoolean(rollupElement.getAttributeValue(ATTR_INCLUDE_DESCRIPTIONS)), - rollupElement.getAttributeValue(ATTR_NAME)); - } - /** * Check equality with another definition. The name is not included in the equality check. - * + * * @param o the other object * @return true if the two definitions describe the same rollup */ @@ -314,13 +173,14 @@ public boolean equals(Object o) { return includeConstituents == that.includeConstituents && includeOriginalColumns == that.includeOriginalColumns && includeDescriptions == that.includeDescriptions && - Objects.equals(groupingColumns, that.groupingColumns) && - Objects.equals(aggregations, that.aggregations); + Objects.equals(aggregations, that.aggregations) && + Objects.equals(groupByColumns, that.groupByColumns); } @Override public int hashCode() { - return Objects.hash(groupingColumns, aggregations, includeConstituents, includeOriginalColumns); + return Objects.hash(aggregations, groupByColumns, + includeConstituents, includeOriginalColumns, includeDescriptions); } public static Builder builder() { @@ -337,8 +197,8 @@ public static class Builder { @SuppressWarnings("MismatchedQueryAndUpdateOfCollection") private Set rollupCache; - private List groupingColumns; - private final Map> aggregations = new LinkedHashMap<>(); + private final List aggregations = new ArrayList<>(); + private List groupByColumns; private boolean includeConstituents; private boolean includeOriginalColumns; private boolean includeDescriptions; @@ -348,7 +208,7 @@ public static class Builder { * Set the name for the rollup. * * @param name the name - * @return this builder + * @return This builder */ public Builder name(String name) { this.name = name; @@ -356,70 +216,43 @@ public Builder name(String name) { } /** - * Set the grouping columns. + * Set the group-by columns. * - * @param columns the grouping columns - * @return this builder + * @param columns the group-by columns + * @return This builder */ - public Builder groupingColumns(String... columns) { - return groupingColumns(Arrays.asList(columns)); + public Builder groupByColumns(String... columns) { + return groupByColumns(Arrays.asList(columns)); } /** - * Set the grouping columns. + * Set the group-by columns. * - * @param columns the grouping columns - * @return this builder + * @param columns the group-by columns + * @return This builder */ - public Builder groupingColumns(Collection columns) { - groupingColumns = new ArrayList<>(columns); + public Builder groupByColumns(Collection columns) { + groupByColumns = new ArrayList<>(columns); return this; } /** - * Set the columns to include for the specified aggregation. + * Add an {@link Aggregation aggregation}. * - * @param type the aggregation - * @param columns the columns to aggregate - * @return this builder + * @param agg The aggregation + * @return This builder */ - public Builder agg(AggType type, String... columns) { - if (columns == null || columns.length == 0) { - aggregations.remove(type); - return this; - } - - return agg(type, Arrays.asList(columns)); - } - - /** - * Set the columns to include for the specified aggregation. - * - * @param type the aggregation - * @param columns the columns to aggregate - * @return this builder - */ - public Builder agg(AggType type, Collection columns) { - if (columns == null || columns.isEmpty()) { - aggregations.remove(type); - return this; - } - - if (type == AggType.Count) { - if (columns.size() > 1) { - throw new IllegalArgumentException("The Count aggregation must have one, and only one column"); - } - } - - aggregations.computeIfAbsent(type, t -> new LinkedHashSet<>()).addAll(columns); + public Builder agg(Aggregation agg) { + aggregations.add(agg); return this; } + /** - * Set if the result table should include constituents. NOTE: This is currently unsupported. + * Set if the result table should include constituents. * - * @param include if constituent rows should be included - * @return this builder + * @param include If constituent rows should be included + * @return This builder */ public Builder includeConstituents(boolean include) { this.includeConstituents = include; @@ -429,8 +262,8 @@ public Builder includeConstituents(boolean include) { /** * Set if the result table should include original columns. NOTE: This is currently unsupported. * - * @param include if original columns should be included - * @return this builder + * @param include If original columns should be included + * @return This builder */ public Builder includeOriginalColumns(boolean include) { this.includeOriginalColumns = include; @@ -440,8 +273,8 @@ public Builder includeOriginalColumns(boolean include) { /** * Set if the rollup should include column descriptions for each column. * - * @param includeDescriptions true if the rollup should add column descriptions - * @return this builder + * @param includeDescriptions Whether the rollup should add column descriptions + * @return This builder */ public Builder withDescriptions(boolean includeDescriptions) { this.includeDescriptions = includeDescriptions; @@ -449,9 +282,9 @@ public Builder withDescriptions(boolean includeDescriptions) { } /** - * Create a RollupDefinition from the state of this builder. + * Create a RollupDefinition from the state of This builder. * - * @return this a new defintion. + * @return this a new definition. * @throws UncheckedDeephavenException if the definition is not complete. */ public RollupDefinition build() throws UncheckedDeephavenException { @@ -459,15 +292,15 @@ public RollupDefinition build() throws UncheckedDeephavenException { throw new UncheckedDeephavenException("Name not defined for rollup"); } - if (groupingColumns == null || groupingColumns.isEmpty()) { - throw new UncheckedDeephavenException("No grouping columns defined"); + if (groupByColumns == null || groupByColumns.isEmpty()) { + throw new UncheckedDeephavenException("No group-by columns defined"); } if (aggregations.isEmpty()) { throw new UncheckedDeephavenException("No aggregations defined"); } - return new RollupDefinition(groupingColumns, aggregations, includeConstituents, includeOriginalColumns, + return new RollupDefinition(aggregations, groupByColumns, includeConstituents, includeOriginalColumns, includeDescriptions, name); } @@ -475,8 +308,7 @@ public RollupDefinition build() throws UncheckedDeephavenException { * Create the rollup definition and attach it to the specified table as a predefined rollup. * * @param attachTo the table to attach to - * @return this builder - * @throws UncheckedDeephavenException + * @return This builder */ public Builder buildAndAttach(Table attachTo) throws UncheckedDeephavenException { return buildAndAttach(attachTo, false); @@ -487,8 +319,7 @@ public Builder buildAndAttach(Table attachTo) throws UncheckedDeephavenException * create and hold a reference to the rollup table if requested. * * @param attachTo the table to attach to - * @return this builder - * @throws UncheckedDeephavenException + * @return This builder */ public Builder buildAndAttach(Table attachTo, boolean preCreate) throws UncheckedDeephavenException { final RollupDefinition def = build(); diff --git a/Plot/src/main/java/io/deephaven/plot/AxesImpl.java b/Plot/src/main/java/io/deephaven/plot/AxesImpl.java index 486aa29140e..af2899d6281 100644 --- a/Plot/src/main/java/io/deephaven/plot/AxesImpl.java +++ b/Plot/src/main/java/io/deephaven/plot/AxesImpl.java @@ -5,8 +5,9 @@ package io.deephaven.plot; import io.deephaven.api.Selectable; +import io.deephaven.api.agg.Aggregation; import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.MemoizedOperationKey; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.plot.axisformatters.AxisFormat; import io.deephaven.plot.axisformatters.NanosAxisFormat; @@ -43,7 +44,6 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.time.DateTime; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.gui.color.Color; import io.deephaven.gui.color.Paint; import io.deephaven.time.calendar.BusinessCalendar; @@ -55,8 +55,8 @@ import java.util.function.Function; import java.util.function.Supplier; +import static io.deephaven.api.agg.Aggregation.AggLast; import static io.deephaven.plot.datasets.interval.IntervalXYDataSeriesArray.*; -import static io.deephaven.engine.table.impl.by.AggregationFactory.AggLast; /** * Chart's axes. @@ -262,14 +262,17 @@ private void registerDataSeries(final SeriesCollection.SeriesType type, final bo } private static SelectableDataSet getAggregatedSelectableDataSet(final SelectableDataSet sds, - final Supplier aggSupplier, final List byColumns) { + final Supplier> aggSupplier, final List byColumns) { final List cols = new ArrayList<>(byColumns); if (sds instanceof SelectableDataSetOneClick) { Collections.addAll(cols, ((SelectableDataSetOneClick) sds).getByColumns()); } - final AggregationFactory caf = aggSupplier.get(); - return sds.transform(caf.getMemoKey(), t -> ((QueryTable) t).by(caf, SelectColumn.from(Selectable.from(cols)))); + final Collection aggs = aggSupplier.get(); + final Collection selectableCols = Selectable.from(cols); + final SelectColumn[] gbsColumns = SelectColumn.from(selectableCols); + final Function applyAggs = t -> t.aggBy(aggs, selectableCols); + return sds.transform(MemoizedOperationKey.aggBy(aggs, gbsColumns), applyAggs); } private static SelectableDataSet getLastBySelectableDataSet(final SelectableDataSet sds, final String... columns) { @@ -1126,7 +1129,7 @@ public CategoryDataSeries catErrorBar(final Comparable seriesName, final Table t public CategoryDataSeries catErrorBar(final Comparable seriesName, final SelectableDataSet sds, final String categories, final String values, final String yLow, final String yHigh) { final SelectableDataSet lastBySelectableDataSet = getAggregatedSelectableDataSet(sds, - () -> PlotUtils.createCategoryComboAgg(AggLast(values, yLow, yHigh)), + () -> PlotUtils.createCategoryAggs(AggLast(values, yLow, yHigh)), Collections.singletonList(categories)); final SwappableTable t = lastBySelectableDataSet.getSwappableTable(seriesName, chart, categories, values, yLow, yHigh, CategoryDataSeries.CAT_SERIES_ORDER_COLUMN); @@ -1178,7 +1181,7 @@ public MultiSeries catErrorBarBy(final Comparable seriesName, final SelectableDa allOfTheByColumns.add(categories); allOfTheByColumns.addAll(Arrays.asList(byColumns)); final SelectableDataSet lastBySelectableDataSet = getAggregatedSelectableDataSet(sds, - () -> PlotUtils.createCategoryComboAgg(AggLast(values, yLow, yHigh)), + () -> PlotUtils.createCategoryAggs(AggLast(values, yLow, yHigh)), allOfTheByColumns); @@ -1707,7 +1710,7 @@ public CategoryDataSeriesInternal catPlot(final Comparable seriesName, final Tab public CategoryDataSeriesInternal catPlot(final Comparable seriesName, final SelectableDataSet sds, final String categories, final String values) { final SelectableDataSet lastBySelectableDataSet = getAggregatedSelectableDataSet(sds, - () -> PlotUtils.createCategoryComboAgg(AggLast(values)), Collections.singletonList(categories)); + () -> PlotUtils.createCategoryAggs(AggLast(values)), Collections.singletonList(categories)); final SwappableTable t = lastBySelectableDataSet.getSwappableTable(seriesName, chart, categories, values, CategoryDataSeries.CAT_SERIES_ORDER_COLUMN); return catPlot( @@ -1759,7 +1762,7 @@ public MultiCatSeriesSwappable catPlotBy(final Comparable seriesName, final Sele allOfTheByColumns.add(categories); allOfTheByColumns.addAll(Arrays.asList(byColumns)); final SelectableDataSet lastBySelectableDataSet = getAggregatedSelectableDataSet(sds, - () -> PlotUtils.createCategoryComboAgg(AggLast(values)), + () -> PlotUtils.createCategoryAggs(AggLast(values)), allOfTheByColumns); final SwappableTable t = lastBySelectableDataSet.getSwappableTable(seriesName, chart, columns); configureCategoryPlot(); @@ -1819,7 +1822,7 @@ public CategoryDataSeriesInternal piePlot(final Comparable seriesName, final Tab public CategoryDataSeriesInternal piePlot(final Comparable seriesName, final SelectableDataSet sds, final String categories, final String values) { final SelectableDataSet lastBySelectableDataSet = getAggregatedSelectableDataSet(sds, - () -> PlotUtils.createCategoryComboAgg(AggLast(values)), Collections.singletonList(categories)); + () -> PlotUtils.createCategoryAggs(AggLast(values)), Collections.singletonList(categories)); final SwappableTable t = lastBySelectableDataSet.getSwappableTable(seriesName, chart, categories, values, CategoryDataSeries.CAT_SERIES_ORDER_COLUMN); return piePlot( diff --git a/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java b/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java index d8331ca3c22..1685303c928 100644 --- a/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java +++ b/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java @@ -5,6 +5,7 @@ package io.deephaven.plot.util; import io.deephaven.api.Selectable; +import io.deephaven.api.agg.Aggregation; import io.deephaven.base.verify.Require; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.table.impl.QueryTable; @@ -19,16 +20,12 @@ import io.deephaven.engine.table.DataColumn; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; -import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.lang.QueryScope; import io.deephaven.time.DateTime; -import io.deephaven.engine.table.impl.by.AggregationFactory; -import io.deephaven.engine.table.impl.by.KeyOnlyFirstOrLastBySpec; import io.deephaven.gui.color.ColorPaletteArray; import io.deephaven.util.QueryConstants; import io.deephaven.engine.util.TableTools; import io.deephaven.engine.table.impl.BaseTable; -import io.deephaven.engine.table.impl.by.AggType; import io.deephaven.engine.table.ColumnSource; import io.deephaven.gui.color.Color; import io.deephaven.gui.color.ColorPalette; @@ -42,8 +39,8 @@ import java.util.function.Function; import java.util.stream.IntStream; +import static io.deephaven.api.agg.Aggregation.AggCount; import static io.deephaven.util.QueryConstants.*; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; import static io.deephaven.function.IntegerNumericPrimitives.abs; /** @@ -726,7 +723,7 @@ public static Table createCategoryTable(final Table t, final String[] catColumns final List lastColumns = t.getDefinition().getColumnNames(); lastColumns.removeAll(Arrays.asList(catColumns)); final Table result = ((QueryTable) t).by( - createCategoryComboAgg(AggLast(lastColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))), + createCategoryAggs(AggLast(lastColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))), SelectColumn.from(Selectable.from(catColumns))); // We must explicitly copy attributes because we are doing a modified manual first/lastBy which will not @@ -736,16 +733,12 @@ public static Table createCategoryTable(final Table t, final String[] catColumns } public static Table createCategoryHistogramTable(final Table t, final String... byColumns) { - return ((QueryTable) t).by(createCategoryComboAgg(AggCount(IntervalXYDataSeriesArray.COUNT)), - SelectColumn.from(Selectable.from(byColumns))); + return t.aggBy(createCategoryAggs(AggCount(IntervalXYDataSeriesArray.COUNT)), Selectable.from(byColumns)); } - public static AggregationFactory createCategoryComboAgg(AggregationElement agg) { - return AggCombo( - Agg(new KeyOnlyFirstOrLastBySpec(CategoryDataSeries.CAT_SERIES_ORDER_COLUMN, AggType.First), - MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY), - agg); + public static Collection createCategoryAggs(Aggregation agg) { + return List.of(Aggregation.AggFirstRowKey(CategoryDataSeries.CAT_SERIES_ORDER_COLUMN), agg); } public static List getColumnConditions(final Table arg, final String column) { diff --git a/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java b/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java index 4a046d6ab15..db990733d3e 100644 --- a/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java +++ b/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java @@ -1,6 +1,7 @@ package io.deephaven.benchmark.engine; import io.deephaven.api.Selectable; +import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.select.SelectColumn; @@ -197,8 +198,7 @@ public Table percentileByStatic(@NotNull final Blackhole bh) { private Function getFunction() { final Function fut; if (percentileMode.equals("normal")) { - fut = (t) -> ((QueryTable) t).by(new PercentileBySpecImpl(0.99), - SelectColumn.from(Selectable.from(keyColumnNames))); + fut = t -> t.aggAllBy(AggSpec.percentile(0.99), SelectColumn.from(Selectable.from(keyColumnNames))); } else if (percentileMode.equals("tdigest")) { fut = (t) -> { final NonKeyColumnAggregationFactory aggregationContextFactory = diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index c681687d94f..c74aa75d4fc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -10,8 +10,7 @@ import io.deephaven.api.Selectable; import io.deephaven.api.SortColumn; import io.deephaven.api.Strings; -import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.AggregationOutputs; +import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecColumnReferences; import io.deephaven.api.filter.Filter; @@ -25,8 +24,6 @@ import io.deephaven.engine.rowset.*; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.by.rollup.NullColumns; -import io.deephaven.engine.table.impl.by.rollup.Partition; import io.deephaven.engine.table.impl.indexer.RowSetIndexer; import io.deephaven.engine.table.impl.perf.BasePerformanceEntry; import io.deephaven.engine.table.impl.perf.QueryPerformanceNugget; @@ -83,8 +80,7 @@ import java.util.stream.Stream; import static io.deephaven.engine.table.MatchPair.matchString; -import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; -import static io.deephaven.engine.table.impl.by.AggregationProcessor.Type.ROLLUP_BASE; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_COLUMN_SUFFIX; /** * Primary coalesced table implementation. @@ -477,39 +473,31 @@ public Table rollup(Collection aggregations, boolean incl final MemoizedOperationKey rollupKey = MemoizedOperationKey.rollup(aggregations, gbsColumns, includeConstituents); return memoizeResult(rollupKey, () -> { - final List baseAggregations = Stream.concat( - aggregations.stream(), - Stream.of(includeConstituents - ? Partition.of(true) - : NullColumns.of(ROLLUP_COLUMN, Object.class)) - ).collect(Collectors.toList()); - final QueryTable baseLevel = aggNoMemo(AggregationProcessor.of(baseAggregations, ROLLUP_BASE), gbsColumns); - - final Deque reaggregateColumns = new ArrayDeque<>(Arrays.asList(gbsColumns)); - final Deque nullColumns = new ArrayDeque<>(groupByColumns.length); + final QueryTable baseLevel = aggNoMemo( + AggregationProcessor.forRollupBase(aggregations, includeConstituents), gbsColumns); + final Deque gbsColumnsToReaggregate = new ArrayDeque<>(Arrays.asList(gbsColumns)); + final Deque nullColumnNames = new ArrayDeque<>(groupByColumns.length); QueryTable lastLevel = baseLevel; - while (!reaggregateColumns.isEmpty()) { - nullColumns.addFirst(reaggregateColumns.removeLast().getName()); - // TODO-RWC: RESUME FROM HERE: FIGURE OUT WHEN TO ADD PARTITION - final Map> nullColumnsMap = new LinkedHashMap<>(nullColumns.size()); - final Table fLastLevel = lastLevel; - nullColumns - .forEach(nc -> nullColumnsMap.put(nc, fLastLevel.getDefinition().getColumn(nc).getDataType())); - - aggregationStateFactory = rollupFactory.withNulls(nullColumnsMap); - lastLevel = lastLevel.byNoMemo(aggregationStateFactory, - reaggregateColumns.toArray(SelectColumn.ZERO_LENGTH_SELECT_COLUMN_ARRAY)); - } - - final String[] rollupsToDrop = lastLevel.getColumnSourceMap().keySet().stream() - .filter(cn -> cn.endsWith(RollupConstants.ROLLUP_COLUMN_SUFFIX)).toArray(String[]::new); - final QueryTable finalTable = (QueryTable) lastLevel.dropColumns(rollupsToDrop); + while (!gbsColumnsToReaggregate.isEmpty()) { + nullColumnNames.addFirst(gbsColumnsToReaggregate.removeLast().getName()); + final TableDefinition lastLevelDefinition = lastLevel.getDefinition(); + final Map> nullColumns = nullColumnNames.stream().collect(Collectors.toMap( + Function.identity(), ncn -> lastLevelDefinition.getColumn(ncn).getDataType(), + Assert::neverInvoked, LinkedHashMap::new)); + lastLevel = lastLevel.aggNoMemo(AggregationProcessor.forRollupReaggregated(aggregations, nullColumns), + gbsColumnsToReaggregate.toArray(SelectColumn.ZERO_LENGTH_SELECT_COLUMN_ARRAY)); + } + + final String[] internalColumnsToDrop = lastLevel.getDefinition().getColumnStream() + .map(ColumnDefinition::getName) + .filter(cn -> cn.endsWith(ROLLUP_COLUMN_SUFFIX)).toArray(String[]::new); + final QueryTable finalTable = (QueryTable) lastLevel.dropColumns(internalColumnsToDrop); final Object reverseLookup = Require.neqNull(lastLevel.getAttribute(REVERSE_LOOKUP_ATTRIBUTE), "REVERSE_LOOKUP_ATTRIBUTE"); finalTable.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, reverseLookup); - final Table result = HierarchicalTable.createFrom(finalTable, new RollupInfo(aggregationFactory, columns, + final Table result = HierarchicalTable.createFrom(finalTable, new RollupInfo(aggregations, gbsColumns, includeConstituents ? RollupInfo.LeafType.Constituent : RollupInfo.LeafType.Normal)); result.setAttribute(Table.HIERARCHICAL_SOURCE_TABLE_ATTRIBUTE, QueryTable.this); copyAttributes(result, CopyAttributeOperation.Rollup); @@ -617,14 +605,20 @@ public Table aggAllBy(AggSpec spec, Selectable... groupByColumns) { throw new IllegalArgumentException( "aggAllBy has no columns to aggregate: spec=" + spec + ", groupByColumns=" + toString(groupByList)); } - final Table tableToUse = AggAllByUseTable.of(this, spec); - final Table result = tableToUse.aggBy(agg.get(), groupByList); - spec.walk(new AggAllByCopyAttributes(this, result)); - return result; + final QueryTable tableToUse = (QueryTable) AggAllByUseTable.of(this, spec); + final List aggs = List.of(agg.get()); + final SelectColumn[] gbsColumns = SelectColumn.from(groupByColumns); + final MemoizedOperationKey aggKey = MemoizedOperationKey.aggBy(aggs, gbsColumns); + return tableToUse.memoizeResult(aggKey, () -> { + final QueryTable result = tableToUse.aggNoMemo(AggregationProcessor.forAggregation(aggs), gbsColumns); + spec.walk(new AggAllByCopyAttributes(this, result)); + return result; + }); } @Override - public Table aggBy(final Collection aggregations, + public Table aggBy( + final Collection aggregations, final Collection groupByColumns) { if (aggregations.isEmpty()) { throw new IllegalArgumentException( @@ -632,25 +626,20 @@ public Table aggBy(final Collection aggregations, + toString(groupByColumns)); } - final List optimized = - AggregationFactory.AggregationElement.optimizeAndConvert(aggregations); - - final List optimizedOrder = optimized.stream() - .map(AggregationFactory.AggregationElement::getResultPairs) - .flatMap(Stream::of) - .map(MatchPair::leftColumn) - .map(ColumnName::of) - .collect(Collectors.toList()); - final List userOrder = AggregationOutputs.of(aggregations).collect(Collectors.toList()); - - final Table aggregationTable = by(new AggregationFactory(optimized), SelectColumn.from(groupByColumns)); + final List optimized = AggregationOptimizer.of(aggregations); + final SelectColumn[] gbsColumns = SelectColumn.from(groupByColumns); + final MemoizedOperationKey aggKey = MemoizedOperationKey.aggBy(optimized, gbsColumns); + final Table aggregationTable = memoizeResult(aggKey, () -> + aggNoMemo(AggregationProcessor.forAggregation(optimized), gbsColumns)); + final List optimizedOrder = AggregationPairs.outputsOf(optimized).collect(Collectors.toList()); + final List userOrder = AggregationPairs.outputsOf(aggregations).collect(Collectors.toList()); if (userOrder.equals(optimizedOrder)) { return aggregationTable; } // We need to re-order the result columns to match the user-provided order - List resultOrder = + final List resultOrder = Stream.concat(groupByColumns.stream().map(Selectable::newColumn), userOrder.stream()) .collect(Collectors.toList()); return aggregationTable.view(resultOrder); @@ -660,12 +649,7 @@ public Table aggBy(final Collection aggregations, public Table countBy(String countColumnName, Selectable... groupByColumns) { return QueryPerformanceRecorder.withNugget( "countBy(" + countColumnName + "," + Arrays.toString(groupByColumns) + ")", sizeForInstrumentation(), - () -> { - if (!COLUMN_NAME.matcher(countColumnName).matches()) { // TODO: Test more columns this way - throw new RuntimeException(countColumnName + " is not a valid column name"); - } - return by(new CountBySpecImpl(countColumnName), SelectColumn.from(groupByColumns)); - }); + () -> aggBy(Aggregation.AggCount(countColumnName), Arrays.asList(groupByColumns))); } // TODO (https://github.com/deephaven/deephaven-core/issues/991): Make this private, and clean up everything that @@ -2903,7 +2887,11 @@ private RowSetBuilderRandom getUngroupIndex( public Table selectDistinct(Collection groupByColumns) { return QueryPerformanceRecorder.withNugget("selectDistinct(" + groupByColumns + ")", sizeForInstrumentation(), - () -> by(new SelectDistinctSpecImpl(), SelectColumn.from(groupByColumns))); + () -> { + final SelectColumn[] gbsColumns = SelectColumn.from(groupByColumns); + final MemoizedOperationKey aggKey = MemoizedOperationKey.aggBy(Collections.emptyList(), gbsColumns); + return memoizeResult(aggKey, () -> aggNoMemo(AggregationProcessor.forSelectDistinct(), gbsColumns)); + }); } @Override diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/RollupInfo.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/RollupInfo.java index d9fa382d525..261a86302f4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/RollupInfo.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/RollupInfo.java @@ -1,7 +1,8 @@ package io.deephaven.engine.table.impl; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.AggregationPairs; import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.select.SelectColumn; import java.util.*; @@ -15,7 +16,7 @@ public class RollupInfo extends AbstractHierarchicalTableInfo { public final LeafType leafType; public final Set byColumnNames; - public final transient AggregationFactory factory; + public final transient Collection aggregations; public final transient SelectColumn[] selectColumns; /** @@ -32,16 +33,16 @@ public enum LeafType { Constituent } - public RollupInfo(AggregationFactory factory, SelectColumn[] selectColumns, LeafType leafType) { - this(factory, selectColumns, leafType, null); + public RollupInfo(Collection aggregations, SelectColumn[] selectColumns, LeafType leafType) { + this(aggregations, selectColumns, leafType, null); } - public RollupInfo(AggregationFactory factory, SelectColumn[] selectColumns, LeafType leafType, + public RollupInfo(Collection aggregations, SelectColumn[] selectColumns, LeafType leafType, String[] columnFormats) { super(columnFormats); - this.factory = factory; + this.aggregations = aggregations; this.selectColumns = selectColumns; - this.matchPairs = factory.getMatchPairs(); + this.matchPairs = AggregationPairs.of(aggregations).map(MatchPair::of).collect(Collectors.toList()); this.leafType = leafType; final Set tempSet = Arrays.stream(selectColumns).map(SelectColumn::getName) @@ -76,7 +77,7 @@ public String getHierarchicalColumnName() { @Override public HierarchicalTableInfo withColumnFormats(String[] columnFormats) { - return new RollupInfo(factory, selectColumns, leafType, columnFormats); + return new RollupInfo(aggregations, selectColumns, leafType, columnFormats); } public LeafType getLeafType() { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 2f96077741a..3cbdde75071 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -4,6 +4,7 @@ import io.deephaven.api.SortColumn; import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.*; +import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.attributes.Values; import io.deephaven.datastructures.util.SmartKey; import io.deephaven.engine.table.*; @@ -11,13 +12,19 @@ import io.deephaven.engine.table.impl.by.rollup.NullColumns; import io.deephaven.engine.table.impl.by.rollup.Partition; import io.deephaven.engine.table.impl.by.rollup.RollupAggregation; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.*; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.*; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.*; import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; +import io.deephaven.engine.table.impl.by.ssmpercentile.SsmChunkedPercentileOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.sources.SingleValueObjectColumnSource; import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; import io.deephaven.time.DateTime; import io.deephaven.util.FunctionalInterfaces.TriFunction; import io.deephaven.util.annotations.FinalDefault; +import io.deephaven.util.type.TypeUtils; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -32,10 +39,16 @@ import java.util.stream.Stream; import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; +import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY; +import static io.deephaven.engine.table.ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY; import static io.deephaven.engine.table.Table.REVERSE_LOOKUP_ATTRIBUTE; +import static io.deephaven.engine.table.impl.RollupAttributeCopier.DEFAULT_INSTANCE; +import static io.deephaven.engine.table.impl.RollupAttributeCopier.LEAF_WITHCONSTITUENTS_INSTANCE; import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; -import static io.deephaven.engine.table.impl.by.IterativeOperatorSpec.*; +import static io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY; import static io.deephaven.engine.table.impl.by.RollupConstants.*; +import static io.deephaven.util.QueryConstants.*; +import static io.deephaven.util.type.TypeUtils.getBoxedType; /** * Conversion tool to generate an {@link AggregationContextFactory} for a collection of {@link Aggregation @@ -43,30 +56,85 @@ */ public class AggregationProcessor implements AggregationContextFactory { + private enum Type { + NORMAL, ROLLUP_BASE, ROLLUP_REAGGREGATED, SELECT_DISTINCT + } + + private final Collection aggregations; + private final Type type; + /** * Convert a collection of {@link Aggregation aggregations} to an {@link AggregationContextFactory}. * - * @param aggregations The {@link Aggregation aggregations} + * @param aggregations The {@link Aggregation aggregations}. Must not be further mutated by the caller. Will not be + * mutated by {@link AggregationProcessor}. * @return The {@link AggregationContextFactory} */ - public static AggregationContextFactory of(@NotNull final Collection aggregations, - @NotNull final Type type) { - return new AggregationProcessor(aggregations, type); + public static AggregationContextFactory forAggregation( + @NotNull final Collection aggregations) { + return new AggregationProcessor(aggregations, Type.NORMAL); } - public enum Type { - NORMAL, ROLLUP_BASE, ROLLUP_REAGGREGATED + /** + * Convert a collection of {@link Aggregation aggregations} to an {@link AggregationContextFactory} for use in + * computing the base level of a rollup. + * + * @param aggregations The {@link Aggregation aggregations}. Must not be further mutated by the caller. Will not be + * mutated by {@link AggregationProcessor}. + * @param includeConstituents Whether constituents should be included via a partition aggregation + * @return The {@link AggregationContextFactory} + */ + public static AggregationContextFactory forRollupBase( + @NotNull final Collection aggregations, + final boolean includeConstituents) { + // @formatter:off + final Collection baseAggregations = + Stream.concat( + aggregations.stream(), + Stream.of(includeConstituents ? Partition.of(true) : NullColumns.of(ROLLUP_COLUMN, Object.class)) + ).collect(Collectors.toList()); + // @formatter:on + return new AggregationProcessor(baseAggregations, Type.ROLLUP_BASE); } - private final Collection aggregations; - private final Type type; + /** + * Convert a collection of {@link Aggregation aggregations} to an {@link AggregationContextFactory} for use in + * computing a reaggregated level of a rollup. + * + * @param aggregations The {@link Aggregation aggregations}. Must not be further mutated by the caller. Will not be + * mutated by {@link AggregationProcessor}. + * @param nullColumns Map of group-by column names and data types to aggregate with a null-column aggregation + * @return The {@link AggregationContextFactory} + */ + public static AggregationContextFactory forRollupReaggregated( + @NotNull final Collection aggregations, + @NotNull final Map> nullColumns) { + // @formatter:off + final Collection reaggregations = + Stream.of( + Stream.of(NullColumns.from(nullColumns)), + aggregations.stream(), + Stream.of(Partition.of(false)) + ).flatMap(Function.identity()).collect(Collectors.toList()); + // @formatter:on + return new AggregationProcessor(reaggregations, Type.ROLLUP_REAGGREGATED); + } + + /** + * Create a trivial {@link AggregationContextFactory} to implement {@link Table#selectDistinct select distinct}. + * + * @return The {@link AggregationContextFactory} + */ + public static AggregationContextFactory forSelectDistinct() { + return new AggregationProcessor(Collections.emptyList(), Type.SELECT_DISTINCT); + } private AggregationProcessor( @NotNull final Collection aggregations, @NotNull final Type type) { this.aggregations = aggregations; this.type = type; - final String duplicationErrorMessage = AggregationOutputs.of(aggregations) + final String duplicationErrorMessage = AggregationPairs.outputsOf(aggregations) .collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream() .filter(kv -> kv.getValue() > 1).map(kv -> kv.getKey() + " used " + kv.getValue() + " times") .collect(Collectors.joining(", ")); @@ -94,6 +162,8 @@ public AggregationContext makeAggregationContext(@NotNull final Table table, return new RollupBaseConverter(table, groupByColumnNames).build(); case ROLLUP_REAGGREGATED: return new RollupReaggregatedConverter(table, groupByColumnNames).build(); + case SELECT_DISTINCT: + return makeEmptyAggregationContext(); default: throw new UnsupportedOperationException("Unsupported type " + type); } @@ -183,6 +253,14 @@ void addNoInputOperator(@NotNull final IterativeChunkedAggregationOperator opera addOperator(operator, null, ZERO_LENGTH_STRING_ARRAY); } + @SafeVarargs + final void addOperator(@NotNull final IterativeChunkedAggregationOperator operator, + @Nullable final ChunkSource.WithPrev inputSource, + @NotNull final Stream... inputColumnNames) { + addOperator(operator, inputSource, + Stream.of(inputColumnNames).flatMap(Function.identity()).toArray(String[]::new)); + } + final void addOperator(@NotNull final IterativeChunkedAggregationOperator operator, @Nullable final ChunkSource.WithPrev inputSource, @NotNull final String... inputColumnNames) { @@ -231,7 +309,7 @@ final void addMinOrMaxOperator(final boolean isMin, @NotNull final String inputN null, inputName); }, () -> addOperator( - getMinMaxChunked(type, resultName, isMin, isAddOnly || isStream), + makeMinOrMaxOperator(type, resultName, isMin, isAddOnly || isStream), inputSource, inputName)); } @@ -279,9 +357,9 @@ final void addSortedFirstOrLastOperator(@NotNull final List sortColu inputSource = TupleSourceFactory.makeTupleSource( Arrays.stream(sortColumnNames).map(table::getColumnSource).toArray(ColumnSource[]::new)); } - // TODO-RWC: Move this helper here or to a new landing place - addOperator(SortedFirstOrLastByAggregationFactory.makeOperator(inputSource.getChunkType(), isFirst, - aggregations.size() > 1, MatchPair.fromPairs(resultPairs), table), + addOperator( + makeSortedFirstOrLastOperator(inputSource.getChunkType(), isFirst, aggregations.size() > 1, + MatchPair.fromPairs(resultPairs), table), inputSource, sortColumnNames); } @@ -292,6 +370,80 @@ final void descendingSortedFirstOrLastUnsupported(@NotNull final SortColumn sort throw new UnsupportedOperationException(String.format("%s does not support sort order in %s", isFirst ? "SortedFirst" : "SortedLast", sortColumn)); } + + final void addWeightedAvgOrSumOperator(@NotNull final String weightName, final boolean isSum) { + final ColumnSource weightSource = table.getColumnSource(weightName); + final boolean weightSourceIsFloatingPoint; + if (isInteger(weightSource.getChunkType())) { + weightSourceIsFloatingPoint = false; + } else if (isFloatingPoint(weightSource.getChunkType())) { + weightSourceIsFloatingPoint = true; + } else { + throw new UnsupportedOperationException( + String.format("Invalid type %s in weight column %s for AggW%s", + weightSource.getType(), weightName, isSum ? "Sum" : "Avg")); + } + + final MutableBoolean anyIntegerResults = new MutableBoolean(); + final MutableBoolean anyFloatingPointResults = new MutableBoolean(); + final List results = resultPairs.stream().map(pair -> { + final ColumnSource inputSource = table.getColumnSource(pair.input().name()); + final ResultType resultType; + if (isInteger(inputSource.getChunkType())) { + if (!weightSourceIsFloatingPoint && isSum) { + anyIntegerResults.setTrue(); + resultType = ResultType.INTEGER; + } else { + anyFloatingPointResults.setTrue(); + resultType = ResultType.FLOATING_POINT; + } + } else if (isFloatingPoint(inputSource.getChunkType())) { + anyFloatingPointResults.setTrue(); + resultType = ResultType.FLOATING_POINT; + } else { + throw new UnsupportedOperationException( + String.format("Invalid type %s in column %s for AggW%s weighted by %s", + inputSource.getType(), pair.input().name(), isSum ? "Sum" : "Avg", weightName)); + } + return new Result(pair, resultType, inputSource); + }).collect(Collectors.toList()); + + final LongWeightRecordingInternalOperator longWeightOperator; + if (anyIntegerResults.booleanValue()) { + longWeightOperator = new LongWeightRecordingInternalOperator(weightSource.getChunkType()); + addOperator(longWeightOperator, weightSource, Stream.of(weightName), + results.stream().filter(r -> r.type == ResultType.INTEGER).map(r -> r.pair.input().name())); + } else { + longWeightOperator = null; + } + + final DoubleWeightRecordingInternalOperator doubleWeightOperator; + if (anyFloatingPointResults.booleanValue()) { + doubleWeightOperator = new DoubleWeightRecordingInternalOperator(weightSource.getChunkType()); + addOperator(doubleWeightOperator, weightSource, Stream.of(weightName), + results.stream().filter(r -> r.type == ResultType.FLOATING_POINT) + .map(r -> r.pair.input().name())); + } else { + doubleWeightOperator = null; + } + + results.forEach(r -> { + final IterativeChunkedAggregationOperator resultOperator; + if (isSum) { + if (r.type == ResultType.INTEGER) { + resultOperator = new LongChunkedWeightedSumOperator( + r.source.getChunkType(), longWeightOperator, r.pair.output().name()); + } else { + resultOperator = new DoubleChunkedWeightedSumOperator( + r.source.getChunkType(), doubleWeightOperator, r.pair.output().name()); + } + } else { + resultOperator = new ChunkedWeightedAverageOperator( + r.source.getChunkType(), doubleWeightOperator, r.pair.output().name()); + } + addOperator(resultOperator, r.source, r.pair.input().name(), weightName); + }); + } } // ----------------------------------------------------------------------------------------------------------------- @@ -333,18 +485,17 @@ public void visit(@NotNull final LastRowKey lastRowKey) { @Override public void visit(@NotNull final AggSpecAbsSum absSum) { - // TODO-RWC: Move this helper and its friends here or to a new landing place - addBasicOperators(IterativeOperatorSpec::getAbsSumChunked); + addBasicOperators((t, n) -> makeSumOperator(t, n, true)); } @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - addBasicOperators((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), false, false)); + addBasicOperators((t, n) -> makeCountDistinctOperator(t, n, countDistinct.countNulls(), false, false)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - addBasicOperators((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), false, false)); + addBasicOperators((t, n) -> makeDistinctOperator(t, n, distinct.includeNulls(), false, false)); } @Override @@ -355,7 +506,7 @@ public void visit(@NotNull final AggSpecGroup group) { @Override public void visit(@NotNull final AggSpecAvg avg) { - addBasicOperators((t, n) -> getAvgChunked(t, n, false)); + addBasicOperators((t, n) -> makeAvgOperator(t, n, false)); } @Override @@ -385,7 +536,7 @@ public void visit(@NotNull final AggSpecMax max) { @Override public void visit(@NotNull final AggSpecMedian median) { - addBasicOperators((t, n) -> getPercentileChunked(t, n, 0.50d, median.averageMedian())); + addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, 0.50d, median.averageMedian(), n)); } @Override @@ -395,7 +546,7 @@ public void visit(@NotNull final AggSpecMin min) { @Override public void visit(@NotNull final AggSpecPercentile pct) { - addBasicOperators((t, n) -> getPercentileChunked(t, n, pct.percentile(), pct.averageMedian())); + addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, pct.percentile(), pct.averageMedian(), n)); } @Override @@ -410,36 +561,33 @@ public void visit(@NotNull final AggSpecSortedLast sortedLast) { @Override public void visit(@NotNull final AggSpecStd std) { - addBasicOperators((t, n) -> getVarChunked(t, n, true, false)); + addBasicOperators((t, n) -> makeVarOrStdOperator(t, n, true, false)); } @Override public void visit(@NotNull final AggSpecSum sum) { - addBasicOperators(IterativeOperatorSpec::getSumChunked); + addBasicOperators((t, n) -> makeSumOperator(t, n, false)); } @Override public void visit(@NotNull final AggSpecUnique unique) { - addBasicOperators((t, n) -> getUniqueChunked(t, n, + addBasicOperators((t, n) -> makeUniqueOperator(t, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), false, false)); } @Override public void visit(@NotNull final AggSpecWAvg wAvg) { - // TODO-RWC: Move this helper here or to a new landing place - WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, wAvg.weight().name(), false, - MatchPair.fromPairs(resultPairs), operators, inputColumnNames, inputSources); + addWeightedAvgOrSumOperator(wAvg.weight().name(), false); } @Override public void visit(@NotNull final AggSpecWSum wSum) { - WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, wSum.weight().name(), true, - MatchPair.fromPairs(resultPairs), operators, inputColumnNames, inputSources); + addWeightedAvgOrSumOperator(wSum.weight().name(), true); } @Override public void visit(@NotNull final AggSpecVar var) { - addBasicOperators((t, n) -> getVarChunked(t, n, false, false)); + addBasicOperators((t, n) -> makeVarOrStdOperator(t, n, false, false)); } } @@ -546,23 +694,19 @@ public void visit(@NotNull final NullColumns nullColumns) { @Override public void visit(@NotNull final Partition partition) { - if (partition.includeConstituents()) { - streamUnsupported("Partition for rollup with constituents included"); + if (!partition.includeConstituents()) { + throw new IllegalArgumentException( + "Partition isn't used for rollup base levels unless constituents are included"); } + streamUnsupported("Partition for rollup with constituents included"); - final QueryTable adjustedTable = partition.includeConstituents() - ? maybeCopyRlAttribute(table, table.updateView(ROLLUP_COLUMN + " = null")) - : table; - final PartitionByChunkedOperator.AttributeCopier copier = partition.includeConstituents() - ? RollupAttributeCopier.LEAF_WITHCONSTITUENTS_INSTANCE - : RollupAttributeCopier.DEFAULT_INSTANCE; + final QueryTable adjustedTable = maybeCopyRlAttribute(table, table.updateView(ROLLUP_COLUMN + " = null")); final PartitionByChunkedOperator partitionOperator = new PartitionByChunkedOperator(table, - adjustedTable, copier, Collections.emptyList(), groupByColumnNames); + adjustedTable, LEAF_WITHCONSTITUENTS_INSTANCE, Collections.emptyList(), groupByColumnNames); addNoInputOperator(partitionOperator); transformers.add(makeRollupKeysTransformer(groupByColumnNames)); - transformers.add(new RollupTableMapAndReverseLookupAttributeSetter(partitionOperator, false, - partition.includeConstituents())); + transformers.add(new RollupTableMapAndReverseLookupAttributeSetter(partitionOperator, false, true)); partitionFound = true; } @@ -573,22 +717,22 @@ public void visit(@NotNull final Partition partition) { @Override public void visit(@NotNull final AggSpecAbsSum absSum) { - addBasicOperators(IterativeOperatorSpec::getAbsSumChunked); + addBasicOperators((t, n) -> makeSumOperator(t, n, true)); } @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - addBasicOperators((t, n) -> getCountDistinctChunked(t, n, countDistinct.countNulls(), true, false)); + addBasicOperators((t, n) -> makeCountDistinctOperator(t, n, countDistinct.countNulls(), true, false)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - addBasicOperators((t, n) -> getDistinctChunked(t, n, distinct.includeNulls(), true, false)); + addBasicOperators((t, n) -> makeDistinctOperator(t, n, distinct.includeNulls(), true, false)); } @Override public void visit(@NotNull final AggSpecAvg avg) { - addBasicOperators((t, n) -> getAvgChunked(t, n, true)); + addBasicOperators((t, n) -> makeAvgOperator(t, n, true)); } @Override @@ -623,29 +767,28 @@ public void visit(@NotNull final AggSpecSortedLast sortedLast) { @Override public void visit(@NotNull final AggSpecStd std) { - addBasicOperators((t, n) -> getVarChunked(t, n, true, true)); + addBasicOperators((t, n) -> makeVarOrStdOperator(t, n, true, true)); } @Override public void visit(@NotNull final AggSpecSum sum) { - addBasicOperators(IterativeOperatorSpec::getSumChunked); + addBasicOperators((t, n) -> makeSumOperator(t, n, false)); } @Override public void visit(@NotNull final AggSpecUnique unique) { - addBasicOperators((t, n) -> getUniqueChunked(t, n, + addBasicOperators((t, n) -> makeUniqueOperator(t, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), true, false)); } @Override public void visit(@NotNull final AggSpecWSum wSum) { - WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, wSum.weight().name(), true, - MatchPair.fromPairs(resultPairs), operators, inputColumnNames, inputSources); + addWeightedAvgOrSumOperator(wSum.weight().name(), true); } @Override public void visit(@NotNull final AggSpecVar var) { - addBasicOperators((t, n) -> getVarChunked(t, n, false, true)); + addBasicOperators((t, n) -> makeVarOrStdOperator(t, n, false, true)); } } @@ -692,7 +835,7 @@ public void visit(@NotNull final Partition partition) { ? table : maybeCopyRlAttribute(table, table.dropColumns(columnsToDrop)); final PartitionByChunkedOperator partitionOperator = new PartitionByChunkedOperator(table, - adjustedTable, RollupAttributeCopier.DEFAULT_INSTANCE, Collections.emptyList(), groupByColumnNames); + adjustedTable, DEFAULT_INSTANCE, Collections.emptyList(), groupByColumnNames); addNoInputOperator(partitionOperator); transformers.add(makeRollupKeysTransformer(groupByColumnNames)); @@ -710,13 +853,13 @@ public void visit(@NotNull final AggSpecAbsSum absSum) { @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> getCountDistinctChunked(ssmType, n, + reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> makeCountDistinctOperator(ssmType, n, countDistinct.countNulls(), true, true)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> getDistinctChunked(priorResultType, n, + reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> makeDistinctOperator(priorResultType, n, distinct.includeNulls(), true, true)); } @@ -767,7 +910,7 @@ public void visit(@NotNull final AggSpecSum sum) { @Override public void visit(@NotNull final AggSpecUnique unique) { - reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> getUniqueChunked( + reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> makeUniqueOperator( priorResultType, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), true, true)); } @@ -786,8 +929,7 @@ private void reaggregateAsSum() { final String resultName = pair.output().name(); final ColumnSource resultSource = table.getColumnSource(resultName); - addOperator(IterativeOperatorSpec.getSumChunked(resultSource.getType(), resultName), - resultSource, resultName); + addOperator(makeSumOperator(resultSource.getType(), resultName, false), resultSource, resultName); } } @@ -950,15 +1092,412 @@ private OP_TYPE getAndAddB } // ----------------------------------------------------------------------------------------------------------------- - // Helpers + // Basic Helpers // ----------------------------------------------------------------------------------------------------------------- + private static AggregationContext makeEmptyAggregationContext() { + // noinspection unchecked + return new AggregationContext( + ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY, + ZERO_LENGTH_STRING_ARRAY_ARRAY, + ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); + } + private static ColumnSource maybeReinterpretDateTimeAsLong(@NotNull final ColumnSource inputSource) { return inputSource.getType() == DateTime.class ? ReinterpretUtils.dateTimeToLongSource(inputSource) : inputSource; } + private static boolean isFloatingPoint(@NotNull final ChunkType chunkType) { + return chunkType == ChunkType.Float + || chunkType == ChunkType.Double; + } + + private static boolean isInteger(@NotNull final ChunkType chunkType) { + return chunkType == ChunkType.Char + || chunkType == ChunkType.Byte + || chunkType == ChunkType.Short + || chunkType == ChunkType.Int + || chunkType == ChunkType.Long; + } + + private enum ResultType { + INTEGER, FLOATING_POINT + } + + private static class Result { + + private final Pair pair; + private final ResultType type; + private final ColumnSource source; + + private Result(@NotNull final Pair pair, @NotNull final ResultType type, + @NotNull final ColumnSource source) { + this.pair = pair; + this.type = type; + this.source = source; + } + } + + // ----------------------------------------------------------------------------------------------------------------- + // Operator Construction Helpers (e.g. to multiplex on input/output data type) + // ----------------------------------------------------------------------------------------------------------------- + + private static IterativeChunkedAggregationOperator makeSumOperator( + @NotNull final Class type, + @NotNull final String name, + final boolean isAbsolute) { + if (type == Boolean.class || type == boolean.class) { + return new BooleanChunkedSumOperator(name); + } else if (type == Byte.class || type == byte.class) { + return new ByteChunkedSumOperator(isAbsolute, name); + } else if (type == Character.class || type == char.class) { + return new CharChunkedSumOperator(isAbsolute, name); + } else if (type == Double.class || type == double.class) { + return new DoubleChunkedSumOperator(isAbsolute, name); + } else if (type == Float.class || type == float.class) { + return new FloatChunkedSumOperator(isAbsolute, name); + } else if (type == Integer.class || type == int.class) { + return new IntChunkedSumOperator(isAbsolute, name); + } else if (type == Long.class || type == long.class) { + return new LongChunkedSumOperator(isAbsolute, name); + } else if (type == Short.class || type == short.class) { + return new ShortChunkedSumOperator(isAbsolute, name); + } else if (type == BigInteger.class) { + return new BigIntegerChunkedSumOperator(isAbsolute, name); + } else if (type == BigDecimal.class) { + return new BigDecimalChunkedSumOperator(isAbsolute, name); + } + throw new UnsupportedOperationException("Unsupported type " + type); + } + + private static IterativeChunkedAggregationOperator makeMinOrMaxOperator( + @NotNull final Class type, + @NotNull final String name, + final boolean isMin, + final boolean isStreamOrAddOnly) { + if (!isStreamOrAddOnly) { + return new SsmChunkedMinMaxOperator(type, isMin, name); + } + if (type == Byte.class || type == byte.class) { + return new ByteChunkedAddOnlyMinMaxOperator(isMin, name); + } else if (type == Character.class || type == char.class) { + return new CharChunkedAddOnlyMinMaxOperator(isMin, name); + } else if (type == Double.class || type == double.class) { + return new DoubleChunkedAddOnlyMinMaxOperator(isMin, name); + } else if (type == Float.class || type == float.class) { + return new FloatChunkedAddOnlyMinMaxOperator(isMin, name); + } else if (type == Integer.class || type == int.class) { + return new IntChunkedAddOnlyMinMaxOperator(isMin, name); + } else if (type == Long.class || type == long.class || type == DateTime.class) { + return new LongChunkedAddOnlyMinMaxOperator(type, isMin, name); + } else if (type == Short.class || type == short.class) { + return new ShortChunkedAddOnlyMinMaxOperator(isMin, name); + } else if (type == Boolean.class || type == boolean.class) { + return new BooleanChunkedAddOnlyMinMaxOperator(isMin, name); + } else { + return new ObjectChunkedAddOnlyMinMaxOperator(type, isMin, name); + } + } + + private static IterativeChunkedAggregationOperator makeCountDistinctOperator( + @NotNull final Class type, + @NotNull final String name, + final boolean countNulls, + final boolean exposeInternal, + final boolean reaggregated) { + if (type == Byte.class || type == byte.class) { + return reaggregated + ? new ByteRollupCountDistinctOperator(name, countNulls) + : new ByteChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else if (type == Character.class || type == char.class) { + return reaggregated + ? new CharRollupCountDistinctOperator(name, countNulls) + : new CharChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else if (type == Double.class || type == double.class) { + return reaggregated + ? new DoubleRollupCountDistinctOperator(name, countNulls) + : new DoubleChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else if (type == Float.class || type == float.class) { + return reaggregated + ? new FloatRollupCountDistinctOperator(name, countNulls) + : new FloatChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else if (type == Integer.class || type == int.class) { + return reaggregated + ? new IntRollupCountDistinctOperator(name, countNulls) + : new IntChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else if (type == Long.class || type == long.class || type == DateTime.class) { + return reaggregated + ? new LongRollupCountDistinctOperator(name, countNulls) + : new LongChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else if (type == Short.class || type == short.class) { + return reaggregated + ? new ShortRollupCountDistinctOperator(name, countNulls) + : new ShortChunkedCountDistinctOperator(name, countNulls, exposeInternal); + } else { + return reaggregated + ? new ObjectRollupCountDistinctOperator(type, name, countNulls) + : new ObjectChunkedCountDistinctOperator(type, name, countNulls, exposeInternal); + } + } + + private static IterativeChunkedAggregationOperator makeDistinctOperator( + @NotNull final Class type, + @NotNull final String name, + final boolean includeNulls, + final boolean exposeInternal, + final boolean reaggregated) { + if (type == Byte.class || type == byte.class) { + return reaggregated + ? new ByteRollupDistinctOperator(name, includeNulls) + : new ByteChunkedDistinctOperator(name, includeNulls, exposeInternal); + } else if (type == Character.class || type == char.class) { + return reaggregated + ? new CharRollupDistinctOperator(name, includeNulls) + : new CharChunkedDistinctOperator(name, includeNulls, exposeInternal); + } else if (type == Double.class || type == double.class) { + return reaggregated + ? new DoubleRollupDistinctOperator(name, includeNulls) + : new DoubleChunkedDistinctOperator(name, includeNulls, exposeInternal); + } else if (type == Float.class || type == float.class) { + return reaggregated + ? new FloatRollupDistinctOperator(name, includeNulls) + : new FloatChunkedDistinctOperator(name, includeNulls, exposeInternal); + } else if (type == Integer.class || type == int.class) { + return reaggregated + ? new IntRollupDistinctOperator(name, includeNulls) + : new IntChunkedDistinctOperator(name, includeNulls, exposeInternal); + } else if (type == Long.class || type == long.class || type == DateTime.class) { + return reaggregated + ? new LongRollupDistinctOperator(type, name, includeNulls) + : new LongChunkedDistinctOperator(type, name, includeNulls, exposeInternal); + } else if (type == Short.class || type == short.class) { + return reaggregated + ? new ShortRollupDistinctOperator(name, includeNulls) + : new ShortChunkedDistinctOperator(name, includeNulls, exposeInternal); + } else { + return reaggregated + ? new ObjectRollupDistinctOperator(type, name, includeNulls) + : new ObjectChunkedDistinctOperator(type, name, includeNulls, exposeInternal); + } + } + + private static IterativeChunkedAggregationOperator makeUniqueOperator( + @NotNull final Class type, + @NotNull final String resultName, + final boolean includeNulls, + @SuppressWarnings("SameParameterValue") final Object onlyNullsSentinel, + final Object nonUniqueSentinel, + final boolean exposeInternal, + final boolean reaggregated) { + checkType(resultName, "Only Nulls Sentinel", type, onlyNullsSentinel); + checkType(resultName, "Non Unique Sentinel", type, nonUniqueSentinel); + + if (type == Byte.class || type == byte.class) { + final byte onsAsType = (onlyNullsSentinel == null) ? NULL_BYTE : ((Number) onlyNullsSentinel).byteValue(); + final byte nusAsType = (nonUniqueSentinel == null) ? NULL_BYTE : ((Number) nonUniqueSentinel).byteValue(); + return reaggregated + ? new ByteRollupUniqueOperator(resultName, includeNulls, onsAsType, nusAsType) + : new ByteChunkedUniqueOperator(resultName, includeNulls, exposeInternal, onsAsType, nusAsType); + } else if (type == Character.class || type == char.class) { + return reaggregated + ? new CharRollupUniqueOperator(resultName, includeNulls, + io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), + io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)) + : new CharChunkedUniqueOperator(resultName, includeNulls, exposeInternal, + io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), + io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)); + } else if (type == Double.class || type == double.class) { + final double onsAsType = + (onlyNullsSentinel == null) ? NULL_DOUBLE : ((Number) onlyNullsSentinel).doubleValue(); + final double nusAsType = + (nonUniqueSentinel == null) ? NULL_DOUBLE : ((Number) nonUniqueSentinel).doubleValue(); + return reaggregated + ? new DoubleRollupUniqueOperator(resultName, includeNulls, onsAsType, nusAsType) + : new DoubleChunkedUniqueOperator(resultName, includeNulls, exposeInternal, onsAsType, nusAsType); + } else if (type == Float.class || type == float.class) { + final float onsAsType = + (onlyNullsSentinel == null) ? NULL_FLOAT : ((Number) onlyNullsSentinel).floatValue(); + final float nusAsType = + (nonUniqueSentinel == null) ? NULL_FLOAT : ((Number) nonUniqueSentinel).floatValue(); + return reaggregated + ? new FloatRollupUniqueOperator(resultName, includeNulls, onsAsType, nusAsType) + : new FloatChunkedUniqueOperator(resultName, includeNulls, exposeInternal, onsAsType, nusAsType); + } else if (type == Integer.class || type == int.class) { + final int onsAsType = (onlyNullsSentinel == null) ? NULL_INT : ((Number) onlyNullsSentinel).intValue(); + final int nusAsType = (nonUniqueSentinel == null) ? NULL_INT : ((Number) nonUniqueSentinel).intValue(); + return reaggregated + ? new IntRollupUniqueOperator(resultName, includeNulls, onsAsType, nusAsType) + : new IntChunkedUniqueOperator(resultName, includeNulls, exposeInternal, onsAsType, nusAsType); + } else if (type == Long.class || type == long.class || type == DateTime.class) { + final long onsAsType; + final long nusAsType; + if (type == DateTime.class) { + onsAsType = (onlyNullsSentinel == null) ? NULL_LONG : ((DateTime) onlyNullsSentinel).getNanos(); + nusAsType = (nonUniqueSentinel == null) ? NULL_LONG : ((DateTime) nonUniqueSentinel).getNanos(); + } else { + onsAsType = (onlyNullsSentinel == null) ? NULL_LONG : ((Number) onlyNullsSentinel).longValue(); + nusAsType = (nonUniqueSentinel == null) ? NULL_LONG : ((Number) nonUniqueSentinel).longValue(); + } + return reaggregated + ? new LongRollupUniqueOperator(type, resultName, includeNulls, onsAsType, nusAsType) + : new LongChunkedUniqueOperator(type, resultName, includeNulls, exposeInternal, onsAsType, + nusAsType); + } else if (type == Short.class || type == short.class) { + final short onsAsType = + (onlyNullsSentinel == null) ? NULL_SHORT : ((Number) onlyNullsSentinel).shortValue(); + final short nusAsType = + (nonUniqueSentinel == null) ? NULL_SHORT : ((Number) nonUniqueSentinel).shortValue(); + return reaggregated + ? new ShortRollupUniqueOperator(resultName, includeNulls, onsAsType, nusAsType) + : new ShortChunkedUniqueOperator(resultName, includeNulls, exposeInternal, onsAsType, nusAsType); + } else { + return reaggregated + ? new ObjectRollupUniqueOperator(type, resultName, includeNulls, onlyNullsSentinel, + nonUniqueSentinel) + : new ObjectChunkedUniqueOperator(type, resultName, includeNulls, exposeInternal, onlyNullsSentinel, + nonUniqueSentinel); + } + } + + private static void checkType(@NotNull final String name, @NotNull final String valueIntent, + @NotNull Class expected, final Object value) { + expected = getBoxedType(expected); + if (value != null && !expected.isAssignableFrom(value.getClass())) { + if (io.deephaven.util.type.TypeUtils.isNumeric(expected) && TypeUtils.isNumeric(value.getClass())) { + if (checkNumericCompatibility((Number) value, expected)) { + return; + } + throw new IllegalArgumentException( + String.format("For result column %s the %s %s is larger than can be represented with a %s", + name, valueIntent, value, expected.getName())); + } + throw new IllegalArgumentException( + String.format("For result column %s the %s must be of type %s but is %s", + name, valueIntent, expected.getName(), value.getClass().getName())); + } + } + + private static boolean checkNumericCompatibility(@NotNull final Number value, @NotNull final Class expected) { + if (expected == Byte.class) { + return Byte.MIN_VALUE <= value.longValue() && value.longValue() <= Byte.MAX_VALUE; + } else if (expected == Short.class) { + return Short.MIN_VALUE <= value.longValue() && value.longValue() <= Short.MAX_VALUE; + } else if (expected == Integer.class) { + return Integer.MIN_VALUE <= value.longValue() && value.longValue() <= Integer.MAX_VALUE; + } else if (expected == Long.class) { + return new BigInteger(value.toString()).compareTo(BigInteger.valueOf(Long.MIN_VALUE)) >= 0 && + new BigInteger(value.toString()).compareTo(BigInteger.valueOf(Long.MAX_VALUE)) <= 0; + } else if (expected == Float.class) { + return value.getClass() != Double.class; + } else if (expected == Double.class) { + return value.getClass() != BigDecimal.class; + } else { + return expected == BigDecimal.class || expected == BigInteger.class; + } + } + + private static IterativeChunkedAggregationOperator makeAvgOperator( + @NotNull final Class type, + @NotNull final String name, + final boolean exposeInternal) { + if (type == Byte.class || type == byte.class) { + return new ByteChunkedAvgOperator(name, exposeInternal); + } else if (type == Character.class || type == char.class) { + return new CharChunkedAvgOperator(name, exposeInternal); + } else if (type == Double.class || type == double.class) { + return new DoubleChunkedAvgOperator(name, exposeInternal); + } else if (type == Float.class || type == float.class) { + return new FloatChunkedAvgOperator(name, exposeInternal); + } else if (type == Integer.class || type == int.class) { + return new IntChunkedAvgOperator(name, exposeInternal); + } else if (type == Long.class || type == long.class) { + return new LongChunkedAvgOperator(name, exposeInternal); + } else if (type == Short.class || type == short.class) { + return new ShortChunkedAvgOperator(name, exposeInternal); + } else if (type == BigInteger.class) { + return new BigIntegerChunkedAvgOperator(name, exposeInternal); + } else if (type == BigDecimal.class) { + return new BigDecimalChunkedAvgOperator(name, exposeInternal); + } else if (AvgState.class.isAssignableFrom(type)) { + throw new UnsupportedOperationException(); + } else if (AvgStateWithNan.class.isAssignableFrom(type)) { + throw new UnsupportedOperationException(); + } + throw new UnsupportedOperationException("Unsupported type " + type); + } + + private static IterativeChunkedAggregationOperator makeVarOrStdOperator( + @NotNull final Class type, + @NotNull final String name, + final boolean isStd, + final boolean exposeInternal) { + if (type == Byte.class || type == byte.class) { + return new ByteChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == Character.class || type == char.class) { + return new CharChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == Double.class || type == double.class) { + return new DoubleChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == Float.class || type == float.class) { + return new FloatChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == Integer.class || type == int.class) { + return new IntChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == Long.class || type == long.class) { + return new LongChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == Short.class || type == short.class) { + return new ShortChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == BigInteger.class) { + return new BigIntegerChunkedVarOperator(isStd, name, exposeInternal); + } else if (type == BigDecimal.class) { + return new BigDecimalChunkedVarOperator(isStd, name, exposeInternal); + } + throw new UnsupportedOperationException("Unsupported type " + type); + } + + static IterativeChunkedAggregationOperator makeSortedFirstOrLastOperator( + @NotNull final ChunkType chunkType, + final boolean isFirst, + final boolean multipleAggs, + @NotNull final MatchPair[] resultPairs, + @NotNull final QueryTable sourceTable) { + if (sourceTable.isAddOnly()) { + // @formatter:off + switch (chunkType) { + case Boolean: throw new UnsupportedOperationException("Columns never use boolean chunks"); + case Char: return new CharAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); + case Byte: return new ByteAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); + case Short: return new ShortAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); + case Int: return new IntAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); + case Long: return new LongAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); + case Float: return new FloatAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); + case Double: return new DoubleAddOnlySortedFirstOrLastChunkedOperator(isFirst, resultPairs, sourceTable, null); + case Object: return new ObjectAddOnlySortedFirstOrLastChunkedOperator(isFirst, resultPairs, sourceTable, null); + } + // @formatter:on + } + if (sourceTable.isStream()) { + // @formatter:off + switch (chunkType) { + case Boolean: throw new UnsupportedOperationException("Columns never use boolean chunks"); + case Char: return new CharStreamSortedFirstOrLastChunkedOperator( isFirst, multipleAggs, resultPairs, sourceTable); + case Byte: return new ByteStreamSortedFirstOrLastChunkedOperator( isFirst, multipleAggs, resultPairs, sourceTable); + case Short: return new ShortStreamSortedFirstOrLastChunkedOperator( isFirst, multipleAggs, resultPairs, sourceTable); + case Int: return new IntStreamSortedFirstOrLastChunkedOperator( isFirst, multipleAggs, resultPairs, sourceTable); + case Long: return new LongStreamSortedFirstOrLastChunkedOperator( isFirst, multipleAggs, resultPairs, sourceTable); + case Float: return new FloatStreamSortedFirstOrLastChunkedOperator( isFirst, multipleAggs, resultPairs, sourceTable); + case Double: return new DoubleStreamSortedFirstOrLastChunkedOperator(isFirst, multipleAggs, resultPairs, sourceTable); + case Object: return new ObjectStreamSortedFirstOrLastChunkedOperator(isFirst, multipleAggs, resultPairs, sourceTable); + } + // @formatter:on + } + return new SortedFirstOrLastChunkedOperator(chunkType, isFirst, resultPairs, sourceTable); + } + + // ----------------------------------------------------------------------------------------------------------------- + // Rollup Structure Helpers + // ----------------------------------------------------------------------------------------------------------------- + private static String makeRedirectionName(final int columnIdentifier) { return ROW_REDIRECTION_PREFIX + columnIdentifier + ROLLUP_COLUMN_SUFFIX; } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java new file mode 100644 index 00000000000..de7ee008dbe --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java @@ -0,0 +1,57 @@ +package io.deephaven.api.agg; + +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * A visitor to describe the input and aggregation {@link Pair column name pairs} for {@link Aggregation aggregations}. + */ +public class AggregationDescriptions implements Aggregation.Visitor { + + public static Map of(Aggregation aggregation) { + return aggregation.walk(new AggregationDescriptions()).getOut(); + } + + public static Map of(Collection aggregations) { + final AggregationDescriptions descriptions = new AggregationDescriptions(); + aggregations.forEach(a -> a.walk(descriptions)); + return descriptions.getOut(); + } + + private final Map out = new LinkedHashMap<>(); + + Map getOut() { + return out; + } + + @Override + public void visit(Count count) { + out.put(count.column().name(), "count"); + } + + @Override + public void visit(FirstRowKey firstRowKey) { + out.put(firstRowKey.column().name(), "first row key"); + } + + @Override + public void visit(LastRowKey lastRowKey) { + out.put(lastRowKey.column().name(), "last row key"); + } + + @Override + public void visit(ColumnAggregation columnAgg) { + visitColumnAgg(columnAgg.pair(), columnAgg.spec().description()); + } + + @Override + public void visit(ColumnAggregations columnAggs) { + final String specDescription = columnAggs.spec().description(); + columnAggs.pairs().forEach(p -> visitColumnAgg(p, specDescription)); + } + + private void visitColumnAgg(Pair pair, String specDescription) { + out.put(pair.output().name(), pair.input().name() + " aggregated with " + specDescription); + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java index dbd332bb224..c098a0d5926 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java @@ -7,46 +7,8 @@ import java.util.stream.Stream; /** - * A visitor to get the ordered output {@link ColumnName column names} for {@link Aggregation aggregations}. + * Helper to get the ordered output {@link ColumnName column names} for {@link Aggregation aggregations}. */ -public class AggregationOutputs implements Aggregation.Visitor { +public class AggregationOutputs { - public static Stream of(Aggregation aggregation) { - return aggregation.walk(new AggregationOutputs()).getOut(); - } - - public static Stream of(Collection aggregations) { - return aggregations.stream().flatMap(AggregationOutputs::of); - } - - private Stream out; - - Stream getOut() { - return Objects.requireNonNull(out); - } - - @Override - public void visit(Count count) { - out = Stream.of(count.column()); - } - - @Override - public void visit(FirstRowKey firstRowKey) { - out = Stream.of(firstRowKey.column()); - } - - @Override - public void visit(LastRowKey lastRowKey) { - out = Stream.of(lastRowKey.column()); - } - - @Override - public void visit(ColumnAggregation columnAgg) { - out = Stream.of(columnAgg.pair().output()); - } - - @Override - public void visit(ColumnAggregations columnAggs) { - out = columnAggs.pairs().stream().map(Pair::output); - } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java new file mode 100644 index 00000000000..d5838dff42b --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java @@ -0,0 +1,60 @@ +package io.deephaven.api.agg; + +import io.deephaven.api.ColumnName; + +import java.util.Collection; +import java.util.Objects; +import java.util.stream.Stream; + +/** + * A visitor to get the ordered input/output {@link Pair column name pairs} for {@link Aggregation aggregations}. + */ +public class AggregationPairs implements Aggregation.Visitor { + + public static Stream of(Aggregation aggregation) { + return aggregation.walk(new AggregationPairs()).getOut(); + } + + public static Stream of(Collection aggregations) { + return aggregations.stream().flatMap(AggregationPairs::of); + } + + public static Stream outputsOf(Aggregation aggregation) { + return of(aggregation).map(Pair::output); + } + + public static Stream outputsOf(Collection aggregations) { + return of(aggregations).map(Pair::output); + } + + private Stream out; + + Stream getOut() { + return Objects.requireNonNull(out); + } + + @Override + public void visit(Count count) { + out = Stream.of(count.column()); + } + + @Override + public void visit(FirstRowKey firstRowKey) { + out = Stream.of(firstRowKey.column()); + } + + @Override + public void visit(LastRowKey lastRowKey) { + out = Stream.of(lastRowKey.column()); + } + + @Override + public void visit(ColumnAggregation columnAgg) { + out = Stream.of(columnAgg.pair()); + } + + @Override + public void visit(ColumnAggregations columnAggs) { + out = columnAggs.pairs().stream(); + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index d14445a0e12..824b55d17b9 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -140,6 +140,8 @@ static AggSpecWSum wsum(String weightColumn) { Aggregation aggregation(Collection pairs); + String description(); + V walk(V visitor); interface Visitor { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java index e0427a8e6c0..3f3a527e974 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java @@ -12,6 +12,11 @@ public static AggSpecAbsSum of() { return ImmutableAggSpecAbsSum.of(); } + @Override + public final String description() { + return "absolute sum"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java index 899fe996db9..ba6737a97bf 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java @@ -11,6 +11,11 @@ public static AggSpecAvg of() { return ImmutableAggSpecAvg.of(); } + @Override + public final String description() { + return "average"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java index 33a28a645d3..45a61621f4d 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java @@ -16,6 +16,11 @@ public static AggSpecCountDistinct of(boolean countNulls) { return ImmutableAggSpecCountDistinct.builder().countNulls(countNulls).build(); } + @Override + public final String description() { + return "count distinct" + (countNulls() ? " (counting nulls)" : ""); + } + @Default public boolean countNulls() { return false; diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java index 8d0df906a5c..388306ccbd1 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java @@ -16,6 +16,11 @@ public static AggSpecDistinct of(boolean includeNulls) { return ImmutableAggSpecDistinct.builder().includeNulls(includeNulls).build(); } + @Override + public final String description() { + return "distinct" + (includeNulls() ? " (including nulls)" : ""); + } + @Default public boolean includeNulls() { return false; diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java index b624f535faa..49b6a3372e7 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java @@ -11,6 +11,11 @@ public static AggSpecFirst of() { return ImmutableAggSpecFirst.of(); } + @Override + public final String description() { + return "first"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java index a8809e1736f..62db0da597e 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java @@ -7,6 +7,8 @@ import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Parameter; +import java.util.stream.Collectors; + @Immutable @BuildableStyle public abstract class AggSpecFormula extends AggSpecBase { @@ -19,6 +21,11 @@ public static AggSpecFormula of(String formula, String formulaParam) { return ImmutableAggSpecFormula.builder().formula(formula).formulaParam(formulaParam).build(); } + @Override + public final String description() { + return "formula '" + formula() + "' with column param '" + formulaParam() + '\''; + } + public abstract String formula(); @Default diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java index 6134dacee5f..ea09f9d670f 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java @@ -11,6 +11,11 @@ public static AggSpecGroup of() { return ImmutableAggSpecGroup.of(); } + @Override + public final String description() { + return "group"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java index 3f2590629f8..7704c140a45 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java @@ -11,6 +11,11 @@ public static AggSpecLast of() { return ImmutableAggSpecLast.of(); } + @Override + public final String description() { + return "last"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java index 1f332aebbeb..820d02cfa6d 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java @@ -11,6 +11,11 @@ public static AggSpecMax of() { return ImmutableAggSpecMax.of(); } + @Override + public final String description() { + return "max"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java index 1cc4a5bd950..378995e287a 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java @@ -16,6 +16,11 @@ public static AggSpecMedian of(boolean averageMedian) { return ImmutableAggSpecMedian.builder().averageMedian(averageMedian).build(); } + @Override + public final String description() { + return "median" + (averageMedian() ? " averaging median" : ""); + } + @Default public boolean averageMedian() { return true; diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java index 9bda455a44d..1146bc297e3 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java @@ -11,6 +11,11 @@ public static AggSpecMin of() { return ImmutableAggSpecMin.of(); } + @Override + public final String description() { + return "min"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java index 161fafd3481..1bc3068a75a 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java @@ -17,6 +17,11 @@ public static AggSpecPercentile of(double percentile, boolean averageMedian) { return ImmutableAggSpecPercentile.builder().percentile(percentile).averageMedian(averageMedian).build(); } + @Override + public final String description() { + return String.format("%.2f percentile%s", percentile(), averageMedian() ? " averaging median" : ""); + } + public abstract double percentile(); @Default diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java index 07429e32f2b..a72d865b9ab 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java @@ -6,6 +6,7 @@ import org.immutables.value.Value.Immutable; import java.util.List; +import java.util.stream.Collectors; /** * Note: the sorted-first aggregation only supports {@link SortColumn.Order#ASCENDING} columns at the moment. @@ -21,6 +22,11 @@ public static Builder builder() { return ImmutableAggSpecSortedFirst.builder(); } + @Override + public final String description() { + return "first sorted by " + columns().stream().map(sc -> sc.column().name()).collect(Collectors.joining(", ")); + } + public abstract List columns(); @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java index e8306b1f8db..fc25de13c92 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java @@ -6,6 +6,7 @@ import org.immutables.value.Value.Immutable; import java.util.List; +import java.util.stream.Collectors; /** * Note: the sorted-last aggregation only supports {@link SortColumn.Order#ASCENDING} columns at the moment. @@ -21,6 +22,11 @@ public static Builder builder() { return ImmutableAggSpecSortedLast.builder(); } + @Override + public final String description() { + return "last sorted by " + columns().stream().map(sc -> sc.column().name()).collect(Collectors.joining(", ")); + } + public abstract List columns(); @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java index 2698bcf18f5..be089079538 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java @@ -11,6 +11,11 @@ public static AggSpecStd of() { return ImmutableAggSpecStd.of(); } + @Override + public final String description() { + return "standard deviation"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java index b33e471913c..40e366cadca 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java @@ -11,6 +11,11 @@ public static AggSpecSum of() { return ImmutableAggSpecSum.of(); } + @Override + public final String description() { + return "sum"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java index 4cf91e8ebbd..96689529e11 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java @@ -38,6 +38,11 @@ public static AggSpecUnique of(boolean includeNulls, Object nonUniqueSentinel) { .build(); } + @Override + public final String description() { + return "unique" + (includeNulls() ? " (including nulls)" : ""); + } + @Default public boolean includeNulls() { return false; diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java index 80d22d75b4d..4a2a95b54a2 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java @@ -11,6 +11,11 @@ public static AggSpecVar of() { return ImmutableAggSpecVar.of(); } + @Override + public final String description() { + return "variance"; + } + @Override public final V walk(V visitor) { visitor.visit(this); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java index 5789950add3..155c0dd88ef 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java @@ -13,6 +13,11 @@ public static AggSpecWAvg of(ColumnName weight) { return ImmutableAggSpecWAvg.of(weight); } + @Override + public final String description() { + return "average weighted by " + weight(); + } + @Parameter public abstract ColumnName weight(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java index c2dcbc265b5..acf010dd9a9 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java @@ -13,6 +13,11 @@ public static AggSpecWSum of(ColumnName weight) { return ImmutableAggSpecWSum.of(weight); } + @Override + public final String description() { + return "sum weighted by " + weight(); + } + @Parameter public abstract ColumnName weight(); From c7bbfe4bc2201d8d6e4aa4b7a2907dfb319e5cba Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 27 Jan 2022 19:28:01 -0500 Subject: [PATCH 17/44] Lots more integration. Need to do ApproximatePercentile now. --- .../table/impl/HierarchicalTableInfo.java | 7 +- .../engine/table/impl/QueryTable.java | 12 +- .../engine/table/impl/by/AggType.java | 2 +- .../table/impl/by/AggregationProcessor.java | 28 +-- .../table/impl/by/ApproximatePercentile.java | 2 +- .../engine/util/TotalsTableBuilder.java | 106 +++++--- .../table/impl/QueryTableAggregationTest.java | 85 +++---- .../engine/table/impl/TestAggBy.java | 228 ++++++++---------- .../by/TestSortedFirstOrLastByFactory.java | 14 +- .../table/ops/ComboAggregateGrpcImpl.java | 121 +++++----- 10 files changed, 285 insertions(+), 320 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/HierarchicalTableInfo.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/HierarchicalTableInfo.java index 8ea629229c4..43e26b58dd9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/HierarchicalTableInfo.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/HierarchicalTableInfo.java @@ -1,15 +1,12 @@ package io.deephaven.engine.table.impl; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.by.AggregationFactory; -import io.deephaven.engine.table.impl.select.SelectColumn; import java.io.Serializable; /** - * A class that contains information required for a particular Hierarchical table type. (i.e - * {@link Table#treeTable(String, String) tree tables} or {@link Table#rollup(AggregationFactory, SelectColumn...) - * rollups}) + * A class that contains information required for a particular Hierarchical table type, for example + * {@link Table#treeTable(String, String) tree tables} or {@link Table#rollup rollups}. */ public interface HierarchicalTableInfo extends Serializable { /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index c74aa75d4fc..a2573aca5c6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -75,7 +75,6 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; -import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -149,8 +148,6 @@ public interface MemoizableOperation> columns; protected transient ModifiedColumnSet modifiedColumnSet; @@ -652,13 +649,6 @@ public Table countBy(String countColumnName, Selectable... groupByColumns) { () -> aggBy(Aggregation.AggCount(countColumnName), Arrays.asList(groupByColumns))); } - // TODO (https://github.com/deephaven/deephaven-core/issues/991): Make this private, and clean up everything that - // uses the AggregationFactory as a specifier. - public Table by(final AggregationSpec inputAggregationSpec, final SelectColumn... groupByColumns) { - return memoizeResult(MemoizedOperationKey.aggBy(inputAggregationSpec, groupByColumns), - () -> byNoMemo(inputAggregationSpec, groupByColumns)); - } - private QueryTable aggNoMemo(@NotNull final AggregationContextFactory aggregationContextFactory, @NotNull final SelectColumn... groupByColumns) { final String description = "aggregation(" + aggregationContextFactory @@ -1607,7 +1597,7 @@ public Table renameColumns(MatchPair... pairs) { for (MatchPair pair : pairs) { if (pair.leftColumn == null || pair.leftColumn.equals("")) { throw new IllegalArgumentException( - "Bad left column in rename pair \"" + pair.toString() + "\""); + "Bad left column in rename pair \"" + pair + "\""); } if (null == columns.get(pair.rightColumn)) { throw new IllegalArgumentException("Column \"" + pair.rightColumn + "\" not found"); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java index 1207d01fab5..94832087ef7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java @@ -3,7 +3,7 @@ import io.deephaven.engine.util.TotalsTableBuilder; /** - * Enumeration representing valid aggregation types for {@link AggregationFactory} or {@link TotalsTableBuilder}. + * Enumeration representing valid aggregation types for {@link TotalsTableBuilder}. */ public enum AggType { /** Return the number of rows in each group. */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 3cbdde75071..0a11c65ef37 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -23,7 +23,6 @@ import io.deephaven.time.DateTime; import io.deephaven.util.FunctionalInterfaces.TriFunction; import io.deephaven.util.annotations.FinalDefault; -import io.deephaven.util.type.TypeUtils; import org.apache.commons.lang3.mutable.MutableBoolean; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -41,6 +40,7 @@ import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY; import static io.deephaven.engine.table.ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY; +import static io.deephaven.engine.table.Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE; import static io.deephaven.engine.table.Table.REVERSE_LOOKUP_ATTRIBUTE; import static io.deephaven.engine.table.impl.RollupAttributeCopier.DEFAULT_INSTANCE; import static io.deephaven.engine.table.impl.RollupAttributeCopier.LEAF_WITHCONSTITUENTS_INSTANCE; @@ -48,7 +48,7 @@ import static io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY; import static io.deephaven.engine.table.impl.by.RollupConstants.*; import static io.deephaven.util.QueryConstants.*; -import static io.deephaven.util.type.TypeUtils.getBoxedType; +import static io.deephaven.util.type.TypeUtils.*; /** * Conversion tool to generate an {@link AggregationContextFactory} for a collection of {@link Aggregation @@ -157,7 +157,7 @@ public AggregationContext makeAggregationContext(@NotNull final Table table, @NotNull final String... groupByColumnNames) { switch (type) { case NORMAL: - return new StandardConverter(table, groupByColumnNames).build(); + return new NormalConverter(table, groupByColumnNames).build(); case ROLLUP_BASE: return new RollupBaseConverter(table, groupByColumnNames).build(); case ROLLUP_REAGGREGATED: @@ -454,9 +454,9 @@ final void addWeightedAvgOrSumOperator(@NotNull final String weightName, final b * Implementation class for conversion from a collection of {@link Aggregation aggregations} to an * {@link AggregationContext} for standard aggregations. Accumulates state by visiting each aggregation. */ - private final class StandardConverter extends Converter { + private final class NormalConverter extends Converter { - private StandardConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { + private NormalConverter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { super(table, groupByColumnNames); } @@ -1303,11 +1303,11 @@ private static IterativeChunkedAggregationOperator makeUniqueOperator( } else if (type == Character.class || type == char.class) { return reaggregated ? new CharRollupUniqueOperator(resultName, includeNulls, - io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), - io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)) + unbox((Character) onlyNullsSentinel), + unbox((Character) nonUniqueSentinel)) : new CharChunkedUniqueOperator(resultName, includeNulls, exposeInternal, - io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), - io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)); + unbox((Character) onlyNullsSentinel), + unbox((Character) nonUniqueSentinel)); } else if (type == Double.class || type == double.class) { final double onsAsType = (onlyNullsSentinel == null) ? NULL_DOUBLE : ((Number) onlyNullsSentinel).doubleValue(); @@ -1365,7 +1365,7 @@ private static void checkType(@NotNull final String name, @NotNull final String @NotNull Class expected, final Object value) { expected = getBoxedType(expected); if (value != null && !expected.isAssignableFrom(value.getClass())) { - if (io.deephaven.util.type.TypeUtils.isNumeric(expected) && TypeUtils.isNumeric(value.getClass())) { + if (isNumeric(expected) && isNumeric(value.getClass())) { if (checkNumericCompatibility((Number) value, expected)) { return; } @@ -1538,9 +1538,9 @@ private RollupTableMapAndReverseLookupAttributeSetter( @Override public QueryTable transformResult(@NotNull final QueryTable table) { - table.setAttribute(QueryTable.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, partitionOperator.getTableMap()); + table.setAttribute(HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, partitionOperator.getTableMap()); if (reaggregated || includeConstituents) { - table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, reverseLookup); + table.setAttribute(REVERSE_LOOKUP_ATTRIBUTE, reverseLookup); } else { setRollupLeafAttributes(table); } @@ -1594,7 +1594,7 @@ public QueryTable transformResult(@NotNull final QueryTable table) { private static void setRollupLeafAttributes(@NotNull final QueryTable table) { table.setAttribute(Table.ROLLUP_LEAF_ATTRIBUTE, RollupInfo.LeafType.Normal); - table.setAttribute(Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, EmptyTableMap.INSTANCE); - table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, ReverseLookup.NULL); + table.setAttribute(HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, EmptyTableMap.INSTANCE); + table.setAttribute(REVERSE_LOOKUP_ATTRIBUTE, ReverseLookup.NULL); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java index e83a6416a53..b0eb9c6bac8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java @@ -223,7 +223,7 @@ private static List flatten(PercentileDefinition value) { return result; } - private void flattenInto(List result) { + private void flattenInto(List result) { if (prior != null) { prior.flattenInto(result); } diff --git a/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java b/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java index c01328de065..9862d4239ed 100644 --- a/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java +++ b/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java @@ -1,18 +1,14 @@ package io.deephaven.engine.util; -import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.api.agg.Aggregation; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.select.SelectColumnFactory; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.by.AggType; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.ColumnSource; import io.deephaven.util.annotations.ScriptApi; import io.deephaven.util.type.EnumValue; import io.deephaven.util.type.TypeUtils; import org.jetbrains.annotations.NotNull; -import java.io.Serializable; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; import java.net.URLEncoder; @@ -20,6 +16,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static io.deephaven.api.agg.Aggregation.*; +import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; import static io.deephaven.engine.table.Table.TOTALS_TABLE_ATTRIBUTE; /** @@ -30,8 +28,46 @@ *

*/ @ScriptApi -public class TotalsTableBuilder implements Serializable { - public static final long serialVersionUID = 1; +public class TotalsTableBuilder { + + /** + * Enumeration representing valid aggregation types for {@link TotalsTableBuilder}. + */ + public enum AggType { + /** Return the number of rows in each group. */ + Count, + /** Return the minimum value of each group. */ + Min, + /** Return the maximum value of each group. */ + Max, + /** Return the sum of values in each group. */ + Sum, + /** Return the sum of absolute values in each group. */ + AbsSum, + /** Return the variance of values in each group. */ + Var, + /** Return the average of values in each group. */ + Avg, + /** Return the standard deviation of each group. */ + Std, + /** Return the first value of each group. */ + First, + /** Return the last value of each group. */ + Last, + /** Return the values of each group as a Vector. */ + Group, + /** Return the number of unique values in each group */ + CountDistinct, + /** Collect the distinct items from the column */ + Distinct, + /** + * Display the singular value from the column if it is unique, or a default value if none are present, or it is not + * unique + */ + Unique, + /** Only valid in a TotalsTableBuilder to indicate we should not perform any aggregation. */ + Skip + } private boolean showTotalsByDefault = false; private boolean showGrandTotalsByDefault = false; @@ -498,11 +534,10 @@ static Table makeTotalsTable(Table source, String aggregationDirective) { * @return an aggregated totals table */ public static Table makeTotalsTable(Table source, TotalsTableBuilder builder, String... groupByColumns) { - final AggregationFactory aggregationFactory = makeAggregationFactory(source, builder); + final Collection aggregations = makeAggregations(source, builder); final String[] formatSpecs = makeColumnFormats(source, builder); - Table totalsTable = ((QueryTable) source.coalesce()) - .by(aggregationFactory, SelectColumnFactory.getExpressions(groupByColumns)); + Table totalsTable = source.aggBy(aggregations, List.of(SelectColumnFactory.getExpressions(groupByColumns))); if (formatSpecs.length > 0) { totalsTable = totalsTable.formatColumns(makeColumnFormats(source, builder)); } @@ -554,19 +589,19 @@ private static String[] makeColumnFormats(Table source, TotalsTableBuilder build } }); - return formatSpecs.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); + return formatSpecs.toArray(ZERO_LENGTH_STRING_ARRAY); } /** - * Produce a AggregationFactory from a source table and builder. + * Produce {@link Aggregation aggregations}from a source table and builder. * * @param source the source table * @param builder the TotalsTableBuilder * - * @return the {@link AggregationFactory} described by source and builder. + * @return the {@link Aggregation aggregations} described by source and builder. */ - public static AggregationFactory makeAggregationFactory(Table source, TotalsTableBuilder builder) { + public static Collection makeAggregations(Table source, TotalsTableBuilder builder) { ensureColumnsExist(source, builder.operationMap.keySet()); final Set defaultOperations = EnumSet.of(builder.defaultOperation); @@ -597,48 +632,41 @@ public static AggregationFactory makeAggregationFactory(Table source, TotalsTabl } } - final List aggregations = new ArrayList<>(); - columnsByType.entrySet().stream().flatMap(e -> makeOperation(e.getKey(), e.getValue())) - .forEach(aggregations::add); - return new AggregationFactory(aggregations); + return columnsByType.entrySet().stream() + .flatMap(e -> makeOperation(e.getKey(), e.getValue().toArray(ZERO_LENGTH_STRING_ARRAY))) + .collect(Collectors.toList()); } - private static Stream makeOperation(AggType operation, List values) { + private static Stream makeOperation(AggType operation, String... columnNames) { switch (operation) { case Group: - throw new IllegalArgumentException("Can not use Array aggregation in totals table."); + throw new IllegalArgumentException("Can not use Group aggregation in totals table."); case Count: - return values.stream().map(AggregationFactory::AggCount); + return Stream.of(columnNames).map(Aggregation::AggCount); case Min: - return Stream.of(AggregationFactory.AggMin(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggMin(columnNames)); case Max: - return Stream.of(AggregationFactory.AggMax(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggMax(columnNames)); case First: - return Stream - .of(AggregationFactory.AggFirst(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggFirst(columnNames)); case Last: - return Stream - .of(AggregationFactory.AggLast(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggLast(columnNames)); case Sum: - return Stream.of(AggregationFactory.AggSum(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggSum(columnNames)); case AbsSum: - return Stream - .of(AggregationFactory.AggAbsSum(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggAbsSum(columnNames)); case Avg: - return Stream.of(AggregationFactory.AggAvg(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggAvg(columnNames)); case Std: - return Stream.of(AggregationFactory.AggStd(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggStd(columnNames)); case Var: - return Stream.of(AggregationFactory.AggVar(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggVar(columnNames)); case Unique: - return Stream - .of(AggregationFactory.AggUnique(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggUnique(columnNames)); case CountDistinct: - return Stream.of(AggregationFactory - .AggCountDistinct(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggCountDistinct(columnNames)); case Distinct: - return Stream - .of(AggregationFactory.AggDistinct(values.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))); + return Stream.of(AggDistinct(columnNames)); default: throw new IllegalStateException(); } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 0fce1a0c95c..16afa5586fa 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -51,6 +51,7 @@ import java.util.stream.Stream; import org.junit.experimental.categories.Category; +import static io.deephaven.api.agg.spec.AggSpec.percentile; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.engine.table.impl.TstUtils.*; @@ -672,10 +673,6 @@ public Table e() { TstUtils.validate(en); } - private static Table by(Table table, AggregationSpec spec, String... groupByColumns) { - return ((QueryTable) table).by(spec, SelectColumn.from(Selectable.from(groupByColumns))); - } - @Test public void testFirstByLastByIncremental() { final Random random = new Random(0); @@ -701,13 +698,9 @@ public void testFirstByLastByIncremental() { new UpdateValidatorNugget(table.sort("Sym", "intCol").firstBy("Sym")), new UpdateValidatorNugget(table.sort("Sym", "intCol").lastBy("Sym")), EvalNugget.from( - () -> by(table.sort("Sym", "intCol"), new TrackingLastBySpecImpl(), "Sym").sort("Sym")), + () -> table.sort("Sym", "intCol").lastBy("Sym").sort("Sym")), EvalNugget.from( - () -> by(table.sort("Sym", "intCol"), new TrackingFirstBySpecImpl(), "Sym").sort("Sym")), - new io.deephaven.engine.table.impl.QueryTableTestBase.TableComparator(table.lastBy("Sym"), - by(table, new TrackingLastBySpecImpl(), "Sym")), - new io.deephaven.engine.table.impl.QueryTableTestBase.TableComparator(table.firstBy("Sym"), - by(table, new TrackingFirstBySpecImpl(), "Sym")), + () -> table.sort("Sym", "intCol").firstBy("Sym").sort("Sym")), EvalNugget.from(() -> table.firstBy("boolCol").sort("boolCol")), EvalNugget.from(() -> table.firstBy("boolCol", "Sym").sort("boolCol", "Sym")), EvalNugget.from(() -> table.firstBy("Sym", "Sym2", "IntSet", "boolCol").sort("Sym", "Sym2", "IntSet", @@ -2299,103 +2292,98 @@ public void testMinMaxByAppend() { } } + private static T setAddOnly(@NotNull final T table) { + table.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, true); + return table; + } + private void testMinMaxByAppend(int size) { final Random random = new Random(0); final ColumnInfo[] columnInfo; - final QueryTable queryTable = getTable(size, random, + final QueryTable queryTable = setAddOnly(getTable(size, random, columnInfo = initColumnInfos(new String[] {"Sym", "intCol", "doubleCol"}, new SetGenerator<>("a", "b", "c", "d"), new IntGenerator(10, 100, 0.1), - new SetGenerator<>(10.1, 20.1, 30.1))); + new SetGenerator<>(10.1, 20.1, 30.1)))); if (RefreshingTableTestCase.printTableUpdates) { TableTools.showWithRowSet(queryTable); } final EvalNuggetInterface[] en = new EvalNuggetInterface[] { new EvalNugget() { public Table e() { - return by(queryTable, new AddOnlyMinMaxBySpecImpl(false), "Sym").sort("Sym"); + return queryTable.maxBy("Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { - return by(queryTable.dropColumns("Sym").update("x = k"), - new AddOnlyMinMaxBySpecImpl(false), "intCol").sort("intCol"); + return setAddOnly(queryTable.dropColumns("Sym").update("x = k")).maxBy("intCol").sort("intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.updateView("x = k"), - new AddOnlyMinMaxBySpecImpl(false), "Sym", "intCol").sort("Sym", "intCol"); + return queryTable.updateView("x = k").maxBy("Sym", "intCol").sort("Sym", "intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1"), new AddOnlyMinMaxBySpecImpl(false), "Sym") - .sort("Sym"); + return setAddOnly(queryTable.update("x=intCol+1")).maxBy("Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1").dropColumns("Sym"), - new AddOnlyMinMaxBySpecImpl(false), "intCol").sort("intCol"); + return setAddOnly(queryTable.update("x=intCol+1").dropColumns("Sym")).maxBy("intCol") + .sort("intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1"), - new AddOnlyMinMaxBySpecImpl(false), "Sym", "intCol").sort("Sym", "intCol"); + return setAddOnly(queryTable.update("x=intCol+1")).maxBy("Sym", "intCol").sort("Sym", "intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1"), new AddOnlyMinMaxBySpecImpl(false), "Sym") - .sort("Sym"); + return setAddOnly(queryTable.update("x=intCol+1")).maxBy("Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { - return by(queryTable, new AddOnlyMinMaxBySpecImpl(true), "Sym").sort("Sym"); + return queryTable.minBy("Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { - return by(queryTable.dropColumns("Sym").update("x = k"), - new AddOnlyMinMaxBySpecImpl(true), "intCol").sort("intCol"); + return setAddOnly(queryTable.dropColumns("Sym").update("x = k")).minBy("intCol").sort("intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.updateView("x = k"), - new AddOnlyMinMaxBySpecImpl(true), "Sym", "intCol").sort("Sym", "intCol"); + return queryTable.updateView("x = k").minBy("Sym", "intCol").sort("Sym", "intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1"), new AddOnlyMinMaxBySpecImpl(true), "Sym") - .sort("Sym"); + return setAddOnly(queryTable.update("x=intCol+1")).minBy("Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1").dropColumns("Sym"), - new AddOnlyMinMaxBySpecImpl(true), "intCol").sort("intCol"); + return setAddOnly(queryTable.update("x=intCol+1").dropColumns("Sym")).minBy("intCol") + .sort("intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1"), - new AddOnlyMinMaxBySpecImpl(true), "Sym", "intCol").sort("Sym", "intCol"); + return setAddOnly(queryTable.update("x=intCol+1")).minBy("Sym", "intCol").sort("Sym", "intCol"); } }, new EvalNugget() { public Table e() { - return by(queryTable.update("x=intCol+1"), new AddOnlyMinMaxBySpecImpl(true), "Sym") - .sort("Sym"); + return setAddOnly(queryTable.update("x=intCol+1")).minBy("Sym").sort("Sym"); } }, - new TableComparator(by(queryTable, new AddOnlyMinMaxBySpecImpl(false), "Sym").sort("Sym"), + new TableComparator(queryTable.maxBy("Sym").sort("Sym"), queryTable.applyToAllBy("max(each)", "Sym").sort("Sym")), - new TableComparator(by(queryTable, new AddOnlyMinMaxBySpecImpl(true), "Sym").sort("Sym"), + new TableComparator(queryTable.minBy("Sym").sort("Sym"), queryTable.applyToAllBy("min(each)", "Sym").sort("Sym")), }; for (int step = 0; step < 50; step++) { @@ -2440,12 +2428,11 @@ private void testMedianByIncremental(int size) { EvalNugget.from(() -> queryTable.view("doubleCol").medianBy()), EvalNugget.Sorted.from(() -> queryTable.medianBy("Sym"), "Sym"), new UpdateValidatorNugget(queryTable.medianBy("Sym")), - EvalNugget.from(() -> by(withoutFloats, new PercentileBySpecImpl(0.25), "Sym").sort("Sym")), - EvalNugget.from(() -> by(withoutFloats, new PercentileBySpecImpl(0.75), "Sym").sort("Sym")), - EvalNugget.from(() -> by(withoutFloats, new PercentileBySpecImpl(0.1), "Sym").sort("Sym")), - EvalNugget.from(() -> by(withoutFloats, new PercentileBySpecImpl(0.99), "Sym").sort("Sym")), - EvalNugget.from(() -> by(withoutFloats.where("Sym=`a`"), new PercentileBySpecImpl(0.99), "Sym") - .sort("Sym")) + EvalNugget.from(() -> withoutFloats.aggAllBy(percentile(0.25), "Sym").sort("Sym")), + EvalNugget.from(() -> withoutFloats.aggAllBy(percentile(0.75), "Sym").sort("Sym")), + EvalNugget.from(() -> withoutFloats.aggAllBy(percentile(0.1), "Sym").sort("Sym")), + EvalNugget.from(() -> withoutFloats.aggAllBy(percentile(0.99), "Sym").sort("Sym")), + EvalNugget.from(() -> withoutFloats.where("Sym=`a`").aggAllBy(percentile(0.99), "Sym").sort("Sym")) }; for (int step = 0; step < 50; step++) { if (RefreshingTableTestCase.printTableUpdates) { @@ -2753,8 +2740,8 @@ public void testMedianTypes() { TableTools.showWithRowSet(table); final Table median = table.medianBy(); - final Table percentile10 = by(table, new PercentileBySpecImpl(0.1)); - final Table percentile90 = by(table, new PercentileBySpecImpl(0.9)); + final Table percentile10 = table.aggAllBy(percentile(0.1)); + final Table percentile90 = table.aggAllBy(percentile(0.9)); TableTools.showWithRowSet(median); TableTools.showWithRowSet(percentile10); TableTools.showWithRowSet(percentile90); @@ -3510,7 +3497,7 @@ public void testIds6332() { col("Value", BigInteger.valueOf(0), new BigInteger("100"), BigInteger.valueOf(100), new BigInteger("100"), new BigInteger("100"), new BigInteger("100"), new BigInteger("100"), new BigInteger("100"), new BigInteger("100"), BigInteger.valueOf(200))); - final Table percentile = by(source, new PercentileBySpecImpl(0.25)); + final Table percentile = source.aggAllBy(percentile(0.25)); TableTools.show(percentile); TestCase.assertEquals(BigInteger.valueOf(100), percentile.getColumn("Value").get(0)); diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java index 547d79661a1..eb80f65cfe8 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java @@ -4,12 +4,10 @@ package io.deephaven.engine.table.impl; -import io.deephaven.api.Selectable; -import io.deephaven.datastructures.util.CollectionUtil; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.engine.table.DataColumn; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.by.*; -import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.vector.CharVector; import io.deephaven.engine.table.lang.QueryLibrary; import io.deephaven.engine.updategraph.UpdateGraphProcessor; @@ -18,22 +16,21 @@ import io.deephaven.test.types.OutOfBandTest; import io.deephaven.util.QueryConstants; import io.deephaven.engine.util.TableTools; -import io.deephaven.engine.table.impl.by.AggregationFormulaSpec; -import io.deephaven.engine.table.impl.by.MinMaxBySpecImpl; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.impl.util.ColumnHolder; -import java.io.IOException; import java.lang.reflect.Array; import java.math.BigInteger; import java.util.Arrays; +import java.util.Collection; +import java.util.List; import java.util.Random; import org.junit.experimental.categories.Category; +import static io.deephaven.api.agg.Aggregation.*; import static io.deephaven.time.DateTimeUtils.convertDateTime; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.engine.table.impl.TstUtils.*; -import static io.deephaven.engine.table.impl.by.AggregationFactory.*; import static io.deephaven.util.QueryConstants.*; import static org.junit.Assert.assertArrayEquals; @@ -45,10 +42,6 @@ protected void setUp() throws Exception { super.setUp(); } - private static Table by(Table table, AggregationSpec spec, String... groupByColumns) { - return ((QueryTable) table).by(spec, SelectColumn.from(Selectable.from(groupByColumns))); - } - public void testBy() { ColumnHolder aHolder = c("A", 0, 0, 1, 1, 0, 0, 1, 1, 0, 0); ColumnHolder bHolder = c("B", 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); @@ -57,14 +50,11 @@ public void testBy() { assertEquals(10, table.size()); assertEquals(2, table.groupBy("A").size()); - AggregationFormulaSpec minFactory = new AggregationFormulaSpec("min(each)", "each"); - AggregationFormulaSpec maxFactory = new AggregationFormulaSpec("max(each)", "each"); - - AggregationFactory minMaxFactory = - new AggregationFactory(new AggregationElementImpl(minFactory, "Min=B"), - new AggregationElementImpl(maxFactory, "Max=B")); - - Table minMax = by(table, minMaxFactory, "A"); + Table minMax = table.aggBy( + List.of( + AggFormula("min(each)", "each", "Min=B"), + AggFormula("max(each)", "each", "Max=B")), + "A"); show(minMax); assertEquals(2, minMax.size()); DataColumn dc = minMax.getColumn("Min"); @@ -74,9 +64,7 @@ public void testBy() { assertEquals(10, dc.get(0)); assertEquals(8, dc.get(1)); - AggregationFactory doubleCountFactory = new AggregationFactory( - new CountAggregationElement("Count1"), new CountAggregationElement("Count2")); - Table doubleCounted = by(table, doubleCountFactory, "A"); + Table doubleCounted = table.aggBy(List.of(AggCount("Count1"), AggCount("Count2")), "A"); show(doubleCounted); assertEquals(2, doubleCounted.size()); @@ -97,7 +85,7 @@ public void testBy() { // minFactory = new AggregationFormulaSpec("min(each)", "each"); // maxFactory = new AggregationFormulaSpec("max(each)", "each"); - AggregationFactory summaryStatisticsFactory = AggCombo( + Collection summaryStatistics = List.of( AggCount("Count"), AggMin("MinB=B", "MinC=C"), AggMed("MedB=B", "MedC=C"), @@ -108,7 +96,7 @@ public void testBy() { AggCountDistinct("DistinctA=A"), AggCountDistinct("DistinctB=B")); - AggregationFactory percentilesFactory = AggCombo( + Collection percentiles = List.of( AggPct(0.25, "Pct01B=B", "Pct01C=C"), AggPct(0.25, "Pct25B=B", "Pct25C=C"), AggPct(0.75, "Pct75B=B", "Pct75C=C"), @@ -127,15 +115,11 @@ public void testBy() { ColumnHolder cHolder = c("C", doubles); table = TableTools.newTable(aHolder, bHolder, cHolder); show(table); - Table summary = by(table, summaryStatisticsFactory, "A"); + Table summary = table.aggBy(summaryStatistics, "A"); show(summary); - // System.out.println("Percentiles (keyed):"); - // Table percentiles = table.groupBy(percentilesFactory, "A"); - // show(percentiles); - System.out.println("\nPercentiles (overall):"); - Table percentilesAll = by(table, percentilesFactory); + Table percentilesAll = table.aggBy(percentiles); show(percentilesAll); } @@ -162,24 +146,15 @@ public void testComboByMinMaxTypes() { new TstUtils.BigDecimalGenerator())); final EvalNuggetInterface[] en = new EvalNuggetInterface[] { - EvalNugget - .from(() -> by(queryTable, AggCombo(AggMin(queryTable.getDefinition().getColumnNamesArray())))), - EvalNugget - .from(() -> by(queryTable, AggCombo(AggMax(queryTable.getDefinition().getColumnNamesArray())))), + EvalNugget.from(() -> queryTable.aggAllBy(AggSpec.min())), + EvalNugget.from(() -> queryTable.aggAllBy(AggSpec.max())), new QueryTableTest.TableComparator( - by(queryTable, AggCombo(AggMin(queryTable.getDefinition().getColumnNamesArray()))), - "AggCombo", - queryTable.minBy(), - "MinBy"), - EvalNugget.Sorted.from( - () -> by(queryTable, AggCombo(AggMin(queryTable.getDefinition().getColumnNamesArray())), "Sym"), - "Sym"), + queryTable.aggAllBy(AggSpec.min()), "aggAllBy", + queryTable.minBy(), "minBy"), + EvalNugget.Sorted.from(() -> queryTable.aggAllBy(AggSpec.min(), "Sym"), "Sym"), new QueryTableTest.TableComparator( - by(queryTable, AggCombo(AggMin(queryTable.getDefinition().getColumnNamesArray())), "Sym") - .sort("Sym"), - "AggCombo", - queryTable.minBy("Sym").sort("Sym"), - "MinBy"), + queryTable.aggAllBy(AggSpec.min(), "Sym").sort("Sym"), "aggAllBy", + queryTable.minBy("Sym").sort("Sym"), "minBy"), }; final int steps = 100; // 8; for (int step = 0; step < steps; step++) { @@ -198,7 +173,7 @@ public void testComboByIncremental() { private void testComboByIncremental(final String ctxt, final int size) { Random random = new Random(0); - ColumnInfo columnInfo[]; + ColumnInfo[] columnInfo; final QueryTable queryTable = getTable(size, random, columnInfo = initColumnInfos(new String[] {"Sym", "intCol", "intColNulls", "doubleCol", "doubleColNulls"}, @@ -210,65 +185,65 @@ private void testComboByIncremental(final String ctxt, final int size) { QueryLibrary.importClass(TestAggBy.class); - final AggregationElement reusedCount = AggCount("Count"); - EvalNuggetInterface en[] = new EvalNuggetInterface[] { + String[] groupByColumns = new String[0]; + EvalNuggetInterface[] en = new EvalNuggetInterface[] { new EvalNugget() { public Table e() { - return by(queryTable, AggCombo(Agg(AggType.Avg, "MeanI=intCol", "MeanD=doubleCol"), - Agg(AggType.Std, "StdI=intCol", "StdD=doubleCol")), "Sym").sort("Sym"); + return queryTable.aggBy(List.of( + AggAvg("MeanI=intCol", "MeanD=doubleCol"), + AggStd("StdI=intCol", "StdD=doubleCol") + ), "Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { - return by(queryTable, AggCombo(AggFormula("min(each)", "each", "MinI=intCol", "MinD=doubleCol"), - AggFormula("max(each)", "each", "MaxI=intCol")), "Sym").sort("Sym"); + return queryTable.aggBy(List.of( + AggFormula("min(each)", "each", "MinI=intCol", "MinD=doubleCol"), + AggFormula("max(each)", "each", "MaxI=intCol") + ), "Sym").sort("Sym"); } }, new QueryTableTest.TableComparator( queryTable.groupBy("Sym").view("Sym", "MinI=min(intCol)", "MinD=min(doubleCol)").sort("Sym"), - "UpdateView", - by(queryTable, - new AggregationFactory( - Agg(new MinMaxBySpecImpl(true), "MinI=intCol", "MinD=doubleCol")), - "Sym").sort("Sym"), - "AggregationElement"), + "view", + queryTable.aggBy(AggMin("MinI=intCol", "MinD=doubleCol"), "Sym").sort("Sym"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym").view("Sym", "MaxI=max(intCol)", "MaxD=max(doubleCol)").sort("Sym"), - "UpdateView", - by(queryTable, AggCombo(Agg(AggType.Max, "MaxI=intCol", "MaxD=doubleCol")), "Sym").sort("Sym"), - "AggregationElement"), + "view", + queryTable.aggBy(AggMax("MaxI=intCol", "MaxD=doubleCol"), "Sym").sort("Sym"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym").view("Sym", "MinI=min(intCol)", "MaxI=max(intCol)").sort("Sym"), - "UpdateView", - by(queryTable, new AggregationFactory(Agg(new MinMaxBySpecImpl(true), "MinI=intCol"), - Agg(new MinMaxBySpecImpl(false), "MaxI=intCol")), "Sym").sort("Sym"), - "AggregationElement"), + "view", + queryTable.aggBy(List.of(AggMin("MinI=intCol"), AggMax("MaxI=intCol")), "Sym").sort("Sym"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym").view("Sym", "MinD=min(doubleCol)", "MaxD=max(doubleCol)").sort("Sym"), - "UpdateView", - by(queryTable, AggCombo(Agg(new MinMaxBySpecImpl(true), "MinD=doubleCol"), - Agg(new MinMaxBySpecImpl(false), "MaxD=doubleCol")), "Sym").sort("Sym"), - "AggregationElement"), + "view", + queryTable.aggBy(List.of(AggMin("MinD=doubleCol"), AggMax("MaxD=doubleCol")), "Sym") + .sort("Sym"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym") .view("Sym", "MinD=min(doubleCol)", "MaxI=max(intCol)", "FirstD=first(doubleCol)", "LastI=last(intCol)") .sort("Sym"), - "UpdateView", - by(queryTable, AggCombo( + "view", + queryTable.aggBy(List.of( AggMin("MinD=doubleCol"), AggMax("MaxI=intCol"), AggFirst("FirstD=doubleCol"), AggLast("LastI=intCol")), "Sym").sort("Sym"), - "AggregationElement"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym") .view("Sym", "MinD=min(doubleCol)", "MaxD=max(doubleCol)", "MinI=min(intCol)", "MaxI=max(intCol)", "LastD=last(doubleCol)", "FirstD=first(doubleCol)", "FirstI=first(intCol)", "LastI=last(intCol)") .sort("Sym"), - "UpdateView", - by(queryTable, AggCombo( + "view", + queryTable.aggBy(List.of( AggMin("MinD=doubleCol"), AggMax("MaxD=doubleCol"), AggMin("MinI=intCol"), @@ -277,14 +252,14 @@ public Table e() { AggFirst("FirstD=doubleCol"), AggFirst("FirstI=intCol"), AggLast("LastI=intCol")), "Sym").sort("Sym"), - "AggregationElement"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy().view("MinD=min(doubleCol)", "MaxI=max(intCol)", "MaxD=max(doubleCol)", "MinI=min(intCol)", "FirstD=first(doubleCol)", "LastI=last(intCol)", "LastD=last(doubleCol)", "FirstI=first(intCol)"), - "UpdateView", - by(queryTable, AggCombo( + "view", + queryTable.aggBy(List.of( AggMin("MinD=doubleCol"), AggMax("MaxI=intCol"), AggMax("MaxD=doubleCol"), @@ -292,21 +267,21 @@ public Table e() { AggFirst("FirstD=doubleCol"), AggLast("LastI=intCol"), AggLast("LastD=doubleCol"), - AggFirst("FirstI=intCol"))), - "AggregationElement"), + AggFirst("FirstI=intCol")), groupByColumns), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym") .view("Sym", "AvgD=avg(doubleCol)", "SumD=sum(doubleCol)", "VarD=var(doubleCol)", "StdD=std(doubleCol)", "intCol") .sort("Sym"), - "UpdateView", - by(queryTable, new AggregationFactory( + "view", + queryTable.aggBy(List.of( AggAvg("AvgD=doubleCol"), AggSum("SumD=doubleCol"), AggVar("VarD=doubleCol"), AggStd("StdD=doubleCol"), AggGroup("intCol")), "Sym").sort("Sym"), - "AggregationElement"), + "aggBy"), new QueryTableTest.TableComparator( queryTable.groupBy("Sym").view("Sym", "MedD=median(doubleCol)", @@ -324,7 +299,7 @@ public Table e() { "Pct90I=(int)TestAggBy.percentile(intCol, 0.90)", "Pct99D=percentile(doubleCol, 0.99)", "Pct99I=(int)TestAggBy.percentile(intCol, 0.99)").sort("Sym"), - by(queryTable, AggCombo( + queryTable.aggBy(List.of( AggMed("MedD=doubleCol"), AggPct(0.01, "Pct01D=doubleCol", "Pct01I=intCol"), AggPct(0.05, "Pct05D=doubleCol", "Pct05I=intCol"), @@ -337,15 +312,15 @@ public Table e() { queryTable.view("Sym", "intCol", "doubleCol").wavgBy("doubleCol", "Sym") .renameColumns("WAvg=intCol"), "WAvgBy", - by(queryTable, AggCombo( + queryTable.aggBy(List.of( AggWAvg("doubleCol", "WAvg=intCol")), "Sym"), "AggWAvg"), new QueryTableTest.TableComparator( queryTable.view("Sym", "intCol", "doubleCol").countBy("Count"), "Count", - by(queryTable, AggCombo(reusedCount), CollectionUtil.ZERO_LENGTH_STRING_ARRAY), "AggCount"), + queryTable.aggBy(AggCount("Count")), "AggCount"), new QueryTableTest.TableComparator( queryTable.view("Sym", "intCol", "doubleCol").countBy("Count"), "Count", - by(queryTable, AggCombo(reusedCount), CollectionUtil.ZERO_LENGTH_STRING_ARRAY), "AggCount"), + queryTable.aggBy(AggCount("Count")), "AggCount"), new QueryTableTestBase.TableComparator( queryTable.groupBy("Sym").view("Sym", "cdi=countDistinct(intCol)", @@ -362,7 +337,7 @@ public Table e() { "uidN=uniqueValue(doubleColNulls, true)") .sort("Sym"), "countDistinctView", - by(queryTable, AggCombo(AggCountDistinct("cdi=intCol", "ddi=doubleCol"), + queryTable.aggBy(List.of(AggCountDistinct("cdi=intCol", "ddi=doubleCol"), AggCountDistinct(true, "cdiN=intColNulls", "ddiN=doubleColNulls"), AggDistinct("dic=intCol", "did=doubleCol"), AggDistinct(true, "dicN=intColNulls", "didN=doubleColNulls"), @@ -380,24 +355,23 @@ public Table e() { } } - public void testComboByDoubleClaim() throws IOException { + public void testComboByDoubleClaim() { final int size = 10; final Random random = new Random(0); - final ColumnInfo columnInfo[]; + final ColumnInfo[] columnInfo; final QueryTable queryTable = getTable(size, random, columnInfo = initColumnInfos(new String[] {"Sym", "intCol", "doubleCol"}, new TstUtils.SetGenerator<>("a", "b", "c", "d"), new TstUtils.IntGenerator(10, 100), new TstUtils.SetGenerator<>(10.1, 20.1, 30.1))); - final AggregationElement reusedCount = AggCount("Count"); - final EvalNuggetInterface en[] = new EvalNuggetInterface[] { + final EvalNuggetInterface[] en = new EvalNuggetInterface[] { new QueryTableTest.TableComparator( queryTable.view("Sym", "intCol", "doubleCol").countBy("Count"), "Count", - by(queryTable, AggCombo(reusedCount), CollectionUtil.ZERO_LENGTH_STRING_ARRAY), "AggCount"), + queryTable.aggBy(AggCount("Count")), "AggCount"), new QueryTableTest.TableComparator( queryTable.view("Sym", "intCol", "doubleCol").countBy("Count"), "Count", - by(queryTable, AggCombo(reusedCount), CollectionUtil.ZERO_LENGTH_STRING_ARRAY), "AggCount") + queryTable.aggBy(AggCount("Count")), "AggCount") }; final int steps = 100; // 8; for (int i = 0; i < steps; i++) { @@ -413,9 +387,8 @@ public void testComboByDistinct() { charCol("Let", 'a', 'b', 'c', 'd')); final Table tail = dataTable.tail(10); - final Table result = by(tail, AggCombo(AggDistinct("Let")), "Grp"); + final Table result = tail.aggBy(AggDistinct("Let"), "Grp"); - // noinspection unchecked final ColumnSource cs = result.getColumnSource("Let"); assertEquals(4, result.size()); assertArrayEquals(new char[] {'a'}, cs.get(0).toArray()); @@ -503,8 +476,8 @@ public void testComboByCountDistinct() { longCol("Account", 1, 1, 2, 1, 3, 2, 4, 2, 5, 5), intCol("Qty", 100, 100, 200, 300, 50, 100, 150, 200, 50, 50)); - Table result = by(dataTable, AggCombo(AggCountDistinct("Account", "Qty")), "USym").sort("USym"); - Table countNulls = by(dataTable, AggCombo(AggCountDistinct(true, "Account", "Qty")), "USym").sort("USym"); + Table result = dataTable.aggBy(AggCountDistinct("Account", "Qty"), "USym").sort("USym"); + Table countNulls = dataTable.aggBy(AggCountDistinct(true, "Account", "Qty"), "USym").sort("USym"); assertEquals(4, result.size()); assertArrayEquals(new Object[] {"AAPL", 2L, 2L}, result.getRecord(0)); assertArrayEquals(new Object[] {"GOOG", 2L, 2L}, result.getRecord(1)); @@ -556,10 +529,9 @@ public void testComboByCountDistinct() { } public void testComboByAggUnique() { - final DateTime dtdefault = convertDateTime("1987-10-20T07:45:00.000 NY"); + final DateTime dtDefault = convertDateTime("1987-10-20T07:45:00.000 NY"); final DateTime dt1 = convertDateTime("2021-01-01T00:00:01.000 NY"); final DateTime dt2 = convertDateTime("2021-01-01T00:00:02.000 NY"); - final DateTime dt3 = convertDateTime("2021-01-01T00:00:03.000 NY"); QueryTable dataTable = TstUtils.testRefreshingTable( c("USym", "AAPL", "AAPL", "AAPL", /**/ "GOOG", "GOOG", /**/ "SPY", "SPY", "SPY", "SPY", /**/ "VXX"), @@ -567,15 +539,17 @@ public void testComboByAggUnique() { intCol("Qty", 100, 100, 100, /**/ 300, 50, /**/ 100, 150, 200, 50, /**/ 50), c("Whee", dt1, dt1, dt1, /**/ dt1, dt2, /**/ dt2, dt2, dt2, dt2, /**/ null)); - Table result = by(dataTable, AggCombo(AggUnique(false, null, -1, "Account", "Qty"), - AggUnique(false, null, dtdefault, "Whee")), "USym").sort("USym"); + Table result = dataTable.aggBy(List.of( + AggUnique(false, -1, "Account", "Qty"), + AggUnique(false, dtDefault, "Whee")), "USym").sort("USym"); - Table countNulls = by(dataTable, AggCombo(AggUnique(true, null, -1, "Account", "Qty"), - AggUnique(true, null, dtdefault, "Whee")), "USym").sort("USym"); + Table countNulls = dataTable.aggBy(List.of( + AggUnique(true, -1, "Account", "Qty"), + AggUnique(true, dtDefault, "Whee")), "USym").sort("USym"); assertEquals(4, result.size()); assertArrayEquals(new Object[] {"AAPL", -1L, 100, dt1}, result.getRecord(0)); - assertArrayEquals(new Object[] {"GOOG", -1L, -1, dtdefault}, result.getRecord(1)); + assertArrayEquals(new Object[] {"GOOG", -1L, -1, dtDefault}, result.getRecord(1)); assertArrayEquals(new Object[] {"SPY", -1L, -1, dt2}, result.getRecord(2)); assertArrayEquals(new Object[] {"VXX", 5L, 50, null}, result.getRecord(3)); assertTableEquals(result, countNulls); @@ -585,12 +559,12 @@ public void testComboByAggUnique() { c("USym", "AAPL", "VXX"), longCol("Account", 1, 5), intCol("Qty", 100, QueryConstants.NULL_INT), - c("Whee", (DateTime) null, (DateTime) null)); + c("Whee", null, (DateTime) null)); dataTable.notifyListeners(i(10), i(), i(2)); }); assertArrayEquals(new Object[] {"AAPL", 1L, 100, dt1}, result.getRecord(0)); - assertArrayEquals(new Object[] {"GOOG", -1L, -1, dtdefault}, result.getRecord(1)); + assertArrayEquals(new Object[] {"GOOG", -1L, -1, dtDefault}, result.getRecord(1)); assertArrayEquals(new Object[] {"SPY", -1L, -1, dt2}, result.getRecord(2)); assertArrayEquals(new Object[] {"VXX", 5L, 50, null}, result.getRecord(3)); @@ -608,11 +582,11 @@ public void testComboByAggUnique() { }); assertArrayEquals(new Object[] {"AAPL", 1L, 100, dt1}, result.getRecord(0)); - assertArrayEquals(new Object[] {"GOOG", -1L, -1, dtdefault}, result.getRecord(1)); + assertArrayEquals(new Object[] {"GOOG", -1L, -1, dtDefault}, result.getRecord(1)); assertArrayEquals(new Object[] {"SPY", -1L, -1, dt2}, result.getRecord(2)); assertArrayEquals(new Object[] {"USO", 2L, 200, dt1}, result.getRecord(3)); - assertArrayEquals(new Object[] {"AAPL", 1L, 100, dtdefault}, countNulls.getRecord(0)); + assertArrayEquals(new Object[] {"AAPL", 1L, 100, dtDefault}, countNulls.getRecord(0)); // UpdateGraphProcessor.DEFAULT.runWithinUnitTestCycle(() -> { @@ -631,13 +605,13 @@ public void testComboByAggUnique() { c("USym", "GOOG", "GOOG", "VXX", "VXX"), longCol("Account", 2L, 2L, QueryConstants.NULL_LONG, 99), intCol("Qty", 350, 350, 50, 50), - c("Whee", dt2, dt2, (DateTime) null, dt1)); + c("Whee", dt2, dt2, null, dt1)); dataTable.notifyListeners(i(9, 10), i(), i(3, 4)); }); assertArrayEquals(new Object[] {"GOOG", 2L, 350, dt2}, result.getRecord(1)); assertArrayEquals(new Object[] {"VXX", 99L, 50, dt1}, result.getRecord(4)); - assertArrayEquals(new Object[] {"VXX", -1L, 50, dtdefault}, countNulls.getRecord(4)); + assertArrayEquals(new Object[] {"VXX", -1L, 50, dtDefault}, countNulls.getRecord(4)); } public void testAggUniqueDefaultValues() { @@ -658,14 +632,12 @@ public void testAggUniqueDefaultValues() { doubleCol("DoubleCol", NULL_DOUBLE, 1.1E22d, 2.2E22d, 3.3E22d)); // First try mixing column types and values - Table result; expectException(IllegalArgumentException.class, "Attempted to use no key/non unique values of incorrect types for aggregated columns!", - () -> by(dataTable, AggCombo(AggUnique(false, -1, -2, "StringCol", "BoolCol", "DatTime", "CharCol", - "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol")), "USym").sort("USym")); + () -> dataTable.aggBy(AggUnique(false, 2, "StringCol", "BoolCol", "DatTime", "CharCol", + "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol"), "USym").sort("USym")); - result = by(dataTable, - AggCombo(AggUnique(false, -1, -2, "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol")), + dataTable.aggBy(AggUnique(false, -2, "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol"), "USym").sort("USym"); // Byte out of range @@ -685,27 +657,20 @@ public void testAggUniqueDefaultValues() { private void testUniqueOutOfRangeParams(Class type, Table dataTable, Number invalidLow, Number validLow, Number invalidHigh, Number validHigh, String... aggCols) { // Byte out of range - expectException(IllegalArgumentException.class, - "Attempted to use no key values too small for " + type.getName() + "!", - () -> by(dataTable, AggCombo(AggUnique(false, invalidLow, -1, aggCols)), "USym").sort("USym")); - - expectException(IllegalArgumentException.class, - "Attempted to use no key values too large for " + type.getName() + "!", - () -> by(dataTable, AggCombo(AggUnique(false, invalidHigh, -1, aggCols)), "USym").sort("USym")); - expectException(IllegalArgumentException.class, "Attempted to non unique values too small for " + type.getName() + "!", - () -> by(dataTable, AggCombo(AggUnique(false, -1, invalidLow, aggCols)), "USym").sort("USym")); + () -> dataTable.aggBy(AggUnique(false, invalidLow, aggCols), "USym").sort("USym")); expectException(IllegalArgumentException.class, "Attempted to use non unique values too large for " + type.getName() + "!", - () -> by(dataTable, AggCombo(AggUnique(false, -1, invalidHigh, aggCols)), "USym").sort("USym")); + () -> dataTable.aggBy(AggUnique(false, invalidHigh, aggCols), "USym").sort("USym")); - by(dataTable, AggCombo(AggUnique(false, validLow, validLow, aggCols)), "USym").sort("USym"); - by(dataTable, AggCombo(AggUnique(false, validHigh, validHigh, aggCols)), "USym").sort("USym"); + dataTable.aggBy(AggUnique(false, validLow, aggCols), "USym").sort("USym"); + dataTable.aggBy(AggUnique(false, validHigh, aggCols), "USym").sort("USym"); } - private static void expectException(Class excType, String failMessage, Runnable action) { + private static void expectException(@SuppressWarnings("SameParameterValue") Class excType, + String failMessage, Runnable action) { try { action.run(); fail(failMessage); @@ -717,7 +682,7 @@ private static void expectException(Class excType, Stri } } - // used in a query test + @SuppressWarnings("unused") // used in a query test public static double percentile(int[] a, double percentile) { if (percentile < 0 || percentile > 1) { throw new RuntimeException("Invalid percentile = " + percentile); @@ -733,5 +698,4 @@ public static double percentile(int[] a, double percentile) { return copy[idx]; } } - } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/by/TestSortedFirstOrLastByFactory.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/by/TestSortedFirstOrLastByFactory.java index 3145b1addfb..29f5290f947 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/by/TestSortedFirstOrLastByFactory.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/by/TestSortedFirstOrLastByFactory.java @@ -4,11 +4,10 @@ package io.deephaven.engine.table.impl.by; +import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.configuration.Configuration; import io.deephaven.engine.table.impl.TableUpdateImpl; -import io.deephaven.engine.table.impl.select.SourceColumn; import io.deephaven.engine.updategraph.UpdateGraphProcessor; -import io.deephaven.engine.util.TableTools; import io.deephaven.engine.util.SortedBy; import io.deephaven.engine.table.impl.*; import io.deephaven.engine.rowset.RowSetFactory; @@ -17,14 +16,14 @@ import junit.framework.TestCase; import java.util.Arrays; +import java.util.List; import java.util.Random; import org.junit.experimental.categories.Category; +import static io.deephaven.api.agg.Aggregation.AggSortedLast; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.engine.table.impl.TstUtils.*; import static io.deephaven.engine.table.impl.TstUtils.addToTable; -import static io.deephaven.engine.table.impl.by.AggregationFactory.AggCombo; -import static io.deephaven.engine.table.impl.by.AggregationFactory.AggSortedLast; @Category(OutOfBandTest.class) public class TestSortedFirstOrLastByFactory extends RefreshingTableTestCase { @@ -76,8 +75,7 @@ private void incrementalTest(int seed, int size, final String... sortColumns) { SortedBy.sortedFirstBy(queryTable, sortColumns, "Sym").sort("Sym")), new QueryTableTest.TableComparator( queryTable.sort(sortColumns).lastBy("Sym").sort("Sym"), - queryTable.by(AggCombo(AggSortedLast(sortColumns, "intCol", "doubleCol", "Indices")), - new SourceColumn("Sym")) + queryTable.aggBy(AggSortedLast(List.of(sortColumns), "intCol", "doubleCol", "Indices"), "Sym") .sort("Sym")) }; for (int step = 0; step < 100; step++) { @@ -98,8 +96,8 @@ public void testIds6445() { // final FuzzerPrintListener pl = new FuzzerPrintListener("source", source); // source.listenForUpdates(pl); - final QueryTable sfb = (QueryTable) source.by(new SortedFirstBy("SFB")); - final QueryTable bucketed = (QueryTable) source.by(new SortedFirstBy("SFB"), new SourceColumn("DummyBucket")); + final QueryTable sfb = (QueryTable) source.aggAllBy(AggSpec.sortedFirst("SFB")); + final QueryTable bucketed = (QueryTable) source.aggAllBy(AggSpec.sortedFirst("SFB"), "DummyBucket"); // final FuzzerPrintListener plsfb = new FuzzerPrintListener("sfb", sfb); // sfb.listenForUpdates(plsfb); diff --git a/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java b/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java index 40b7b5b6cbf..bdab79b79de 100644 --- a/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java @@ -1,12 +1,11 @@ package io.deephaven.server.table.ops; import com.google.rpc.Code; +import io.deephaven.api.agg.Aggregation; import io.deephaven.base.verify.Assert; import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.DataColumn; +import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.by.AggregationFactory; import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.engine.table.impl.select.SelectColumnFactory; import io.deephaven.extensions.barrage.util.GrpcUtil; @@ -15,15 +14,19 @@ import io.deephaven.server.session.SessionState; import io.deephaven.server.table.validation.ColumnExpressionValidator; import io.grpc.StatusRuntimeException; +import org.jetbrains.annotations.NotNull; import javax.inject.Inject; import javax.inject.Singleton; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Set; -import java.util.function.Supplier; +import java.util.function.Function; import java.util.stream.Collectors; +import static io.deephaven.api.agg.Aggregation.*; + @Singleton public class ComboAggregateGrpcImpl extends GrpcTableOperation { @@ -156,66 +159,64 @@ private static Table comboAggregateHelper(final Table parent, final SelectColumn final List aggregates) { final Set groupByColumnSet = Arrays.stream(groupByColumns).map(SelectColumn::getName).collect(Collectors.toSet()); + final Function getPairs = + agg -> getColumnPairs(parent, groupByColumnSet, agg); - final AggregationFactory.AggregationElement[] aggregationElement = - new AggregationFactory.AggregationElement[aggregates.size()]; - - for (int i = 0; i < aggregates.size(); i++) { - final ComboAggregateRequest.Aggregate agg = aggregates.get(i); - - final String[] matchPairs; - if (agg.getMatchPairsCount() == 0) { - // if not specified, we apply the aggregate to all columns not "otherwise involved" - matchPairs = Arrays.stream(parent.getColumns()) - .map(DataColumn::getName) - .filter(n -> !(groupByColumnSet.contains(n) - || (agg.getType() == ComboAggregateRequest.AggType.WEIGHTED_AVG - && agg.getColumnName().equals(n)))) - .toArray(String[]::new); - } else { - matchPairs = agg.getMatchPairsList().toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); - final SelectColumn[] matchPairExpressions = SelectColumnFactory.getExpressions(matchPairs); - ColumnExpressionValidator.validateColumnExpressions(matchPairExpressions, matchPairs, parent); - } + final Collection aggregations = aggregates.stream().map( + agg -> makeAggregation(agg, getPairs) + ).collect(Collectors.toList()); - final Supplier comboMapper = () -> { - switch (agg.getType()) { - case SUM: - return AggregationFactory.AggSum(matchPairs); - case ABS_SUM: - return AggregationFactory.AggAbsSum(matchPairs); - case GROUP: - return AggregationFactory.AggGroup(matchPairs); - case AVG: - return AggregationFactory.AggAvg(matchPairs); - case COUNT: - return AggregationFactory.AggCount(agg.getColumnName()); - case FIRST: - return AggregationFactory.AggFirst(matchPairs); - case LAST: - return AggregationFactory.AggLast(matchPairs); - case MIN: - return AggregationFactory.AggMin(matchPairs); - case MAX: - return AggregationFactory.AggMax(matchPairs); - case MEDIAN: - return AggregationFactory.AggMed(matchPairs); - case PERCENTILE: - return AggregationFactory.AggPct(agg.getPercentile(), agg.getAvgMedian(), matchPairs); - case STD: - return AggregationFactory.AggStd(matchPairs); - case VAR: - return AggregationFactory.AggVar(matchPairs); - case WEIGHTED_AVG: - return AggregationFactory.AggWAvg(agg.getColumnName(), matchPairs); - default: - throw new UnsupportedOperationException("Unsupported aggregate: " + agg.getType()); - } - }; + return parent.aggBy(aggregations, Arrays.asList(groupByColumns)); + } - aggregationElement[i] = comboMapper.get(); + private static String[] getColumnPairs(@NotNull final Table parent, + @NotNull final Set groupByColumnSet, + @NotNull final ComboAggregateRequest.Aggregate agg) { + if (agg.getMatchPairsCount() == 0) { + // If not specified, we apply the aggregate to all columns not "otherwise involved" + return parent.getDefinition().getColumnStream() + .map(ColumnDefinition::getName) + .filter(n -> !(groupByColumnSet.contains(n) || + (agg.getType() == ComboAggregateRequest.AggType.WEIGHTED_AVG && agg.getColumnName().equals(n)))) + .toArray(String[]::new); } + return agg.getMatchPairsList().toArray(String[]::new); + } - return ((QueryTable) parent).by(AggregationFactory.AggCombo(aggregationElement), groupByColumns); + private static Aggregation makeAggregation( + @NotNull final ComboAggregateRequest.Aggregate agg, + @NotNull final Function getPairs) { + switch (agg.getType()) { + case SUM: + return AggSum(getPairs.apply(agg)); + case ABS_SUM: + return AggAbsSum(getPairs.apply(agg)); + case GROUP: + return AggGroup(getPairs.apply(agg)); + case AVG: + return Aggregation.AggAvg(getPairs.apply(agg)); + case COUNT: + return Aggregation.AggCount(agg.getColumnName()); + case FIRST: + return Aggregation.AggFirst(getPairs.apply(agg)); + case LAST: + return Aggregation.AggLast(getPairs.apply(agg)); + case MIN: + return Aggregation.AggMin(getPairs.apply(agg)); + case MAX: + return Aggregation.AggMax(getPairs.apply(agg)); + case MEDIAN: + return Aggregation.AggMed(getPairs.apply(agg)); + case PERCENTILE: + return Aggregation.AggPct(agg.getPercentile(), agg.getAvgMedian(), getPairs.apply(agg)); + case STD: + return Aggregation.AggStd(getPairs.apply(agg)); + case VAR: + return Aggregation.AggVar(getPairs.apply(agg)); + case WEIGHTED_AVG: + return Aggregation.AggWAvg(agg.getColumnName(), getPairs.apply(agg)); + default: + throw new UnsupportedOperationException("Unsupported aggregate: " + agg.getType()); + } } } From 0e22f00d547b12410c914dfd0575d9c0d466681a Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 27 Jan 2022 22:02:51 -0500 Subject: [PATCH 18/44] Re-name some stuff. --- .../table/impl/by/AggregationProcessor.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 0a11c65ef37..977b94958a7 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -386,33 +386,34 @@ final void addWeightedAvgOrSumOperator(@NotNull final String weightName, final b final MutableBoolean anyIntegerResults = new MutableBoolean(); final MutableBoolean anyFloatingPointResults = new MutableBoolean(); - final List results = resultPairs.stream().map(pair -> { + final List results = resultPairs.stream().map(pair -> { final ColumnSource inputSource = table.getColumnSource(pair.input().name()); - final ResultType resultType; + final WeightedOpResultType resultType; if (isInteger(inputSource.getChunkType())) { if (!weightSourceIsFloatingPoint && isSum) { anyIntegerResults.setTrue(); - resultType = ResultType.INTEGER; + resultType = WeightedOpResultType.INTEGER; } else { anyFloatingPointResults.setTrue(); - resultType = ResultType.FLOATING_POINT; + resultType = WeightedOpResultType.FLOATING_POINT; } } else if (isFloatingPoint(inputSource.getChunkType())) { anyFloatingPointResults.setTrue(); - resultType = ResultType.FLOATING_POINT; + resultType = WeightedOpResultType.FLOATING_POINT; } else { throw new UnsupportedOperationException( String.format("Invalid type %s in column %s for AggW%s weighted by %s", inputSource.getType(), pair.input().name(), isSum ? "Sum" : "Avg", weightName)); } - return new Result(pair, resultType, inputSource); + return new WeightedOpResult(pair, resultType, inputSource); }).collect(Collectors.toList()); final LongWeightRecordingInternalOperator longWeightOperator; if (anyIntegerResults.booleanValue()) { longWeightOperator = new LongWeightRecordingInternalOperator(weightSource.getChunkType()); addOperator(longWeightOperator, weightSource, Stream.of(weightName), - results.stream().filter(r -> r.type == ResultType.INTEGER).map(r -> r.pair.input().name())); + results.stream().filter(r -> r.type == WeightedOpResultType.INTEGER) + .map(r -> r.pair.input().name())); } else { longWeightOperator = null; } @@ -421,7 +422,7 @@ final void addWeightedAvgOrSumOperator(@NotNull final String weightName, final b if (anyFloatingPointResults.booleanValue()) { doubleWeightOperator = new DoubleWeightRecordingInternalOperator(weightSource.getChunkType()); addOperator(doubleWeightOperator, weightSource, Stream.of(weightName), - results.stream().filter(r -> r.type == ResultType.FLOATING_POINT) + results.stream().filter(r -> r.type == WeightedOpResultType.FLOATING_POINT) .map(r -> r.pair.input().name())); } else { doubleWeightOperator = null; @@ -430,7 +431,7 @@ final void addWeightedAvgOrSumOperator(@NotNull final String weightName, final b results.forEach(r -> { final IterativeChunkedAggregationOperator resultOperator; if (isSum) { - if (r.type == ResultType.INTEGER) { + if (r.type == WeightedOpResultType.INTEGER) { resultOperator = new LongChunkedWeightedSumOperator( r.source.getChunkType(), longWeightOperator, r.pair.output().name()); } else { @@ -1122,18 +1123,18 @@ private static boolean isInteger(@NotNull final ChunkType chunkType) { || chunkType == ChunkType.Long; } - private enum ResultType { + private enum WeightedOpResultType { INTEGER, FLOATING_POINT } - private static class Result { + private static class WeightedOpResult { private final Pair pair; - private final ResultType type; + private final WeightedOpResultType type; private final ColumnSource source; - private Result(@NotNull final Pair pair, @NotNull final ResultType type, - @NotNull final ColumnSource source) { + private WeightedOpResult(@NotNull final Pair pair, @NotNull final WeightedOpResultType type, + @NotNull final ColumnSource source) { this.pair = pair; this.type = type; this.source = source; From 4972054049c6ef9ff380582c6f0c4e30817f37ba Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 27 Jan 2022 22:06:34 -0500 Subject: [PATCH 19/44] Delete extra whitespace. --- .../deephaven/engine/table/impl/by/ApproximatePercentile.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java index b0eb9c6bac8..e83a6416a53 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java @@ -223,7 +223,7 @@ private static List flatten(PercentileDefinition value) { return result; } - private void flattenInto(List result) { + private void flattenInto(List result) { if (prior != null) { prior.flattenInto(result); } From 171b5f4514338660e32b200ec6af96af70481866 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 27 Jan 2022 23:55:34 -0500 Subject: [PATCH 20/44] WIP on ApproximatePercentile --- .../io/deephaven/annotations/TupleStyle.java | 24 ++++ .../impl/by/AggregationElementAdapter.java | 6 + .../table/impl/by/AggregationProcessor.java | 21 +++ .../table/impl/by/ApproximatePercentile.java | 5 + .../client/impl/BatchTableRequestBuilder.java | 6 + .../io/deephaven/api/agg/Aggregation.java | 5 + .../api/agg/AggregationDescriptions.java | 9 +- .../api/agg/AggregationOptimizer.java | 7 + .../deephaven/api/agg/AggregationPairs.java | 11 ++ .../api/agg/ApproximatePercentile.java | 125 ++++++++++++++++++ 10 files changed, 218 insertions(+), 1 deletion(-) create mode 100644 Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java diff --git a/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java b/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java new file mode 100644 index 00000000000..41799f332c5 --- /dev/null +++ b/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java @@ -0,0 +1,24 @@ +package io.deephaven.annotations; + +import org.immutables.value.Value; +import org.immutables.value.Value.Style.ImplementationVisibility; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * A tuple style is for objects that represent simple tuples, with all parameters specified at construction. Not recommended for objects with more than two fields. Not + * applicable for objects with default fields. + */ +@Target({ElementType.TYPE, ElementType.PACKAGE}) +@Retention(RetentionPolicy.CLASS) +@Value.Style(visibility = ImplementationVisibility.PACKAGE, + allParameters = true, + typeImmutable = "*Tuple", + defaults = @Value.Immutable(builder = false, copy = false), strictBuilder = true, + weakInterning = true, jdkOnly = true) +public @interface TupleStyle { + // Note: this produces XTuple.of() methods for the implementation classes +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java index 9706f9a330b..d102f5f5d02 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl.by; import io.deephaven.api.agg.*; +import io.deephaven.api.agg.ApproximatePercentile; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElementImpl; @@ -41,6 +42,11 @@ public void visit(LastRowKey lastRowKey) { MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY); } + @Override + public void visit(ApproximatePercentile approximatePercentile) { + // WHO CARE I'm DELETING THIS CLASS + } + @Override public void visit(ColumnAggregation columnAgg) { final AggregationSpec spec = AggregationSpecAdapter.of(columnAgg.spec()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 977b94958a7..d9be876c59f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -3,6 +3,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; import io.deephaven.api.agg.*; +import io.deephaven.api.agg.ApproximatePercentile; import io.deephaven.api.agg.spec.*; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.attributes.Values; @@ -480,6 +481,21 @@ public void visit(@NotNull final LastRowKey lastRowKey) { addFirstOrLastOperators(false, lastRowKey.column().name()); } + @Override + public void visit(ApproximatePercentile approximatePercentile) { + final String inputName = approximatePercentile.input().name(); + final ColumnSource inputSource = table.getColumnSource(inputName); + final IterativeChunkedAggregationOperator operator = new TDigestPercentileOperator( + inputSource.getType(), + approximatePercentile.compression(), + approximatePercentile.digest().map(ColumnName::name).orElse(null), + approximatePercentile.percentileOutputs().stream() + .mapToDouble(ApproximatePercentile.PercentileOutput::percentile).toArray(), + approximatePercentile.percentileOutputs().stream() + .map(po -> po.output().name()).toArray(String[]::new)); + addOperator(operator, inputSource, inputName); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -614,6 +630,11 @@ default void visit(@NotNull final LastRowKey lastRowKey) { rollupUnsupported("LastRowKey"); } + @Override + default void visit(ApproximatePercentile approximatePercentile) { + rollupUnsupported("ApproximatePercentile"); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor for unsupported column aggregation specs // ------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java index e83a6416a53..37d14037cc4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java @@ -246,6 +246,11 @@ public static Table approximatePercentiles(Table input, PercentileDefinition per throw new IllegalArgumentException("No percentile columns defined!"); } final double compression = flatDefs.get(0).compression; + + io.deephaven.api.agg.ApproximatePercentile.Builder aggBuilder = + io.deephaven.api.agg.ApproximatePercentile.builder(); + aggBuilder.input(flatDefs) + final NonKeyColumnAggregationFactory aggregationContextFactory = new NonKeyColumnAggregationFactory((type, resultName, exposeInternalColumns) -> { for (final PercentileDefinition percentileDefinition : flatDefs) { diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index 397740e272a..d25d814a83e 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -544,6 +544,12 @@ public void visit(LastRowKey lastRowKey) { "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } + @Override + public void visit(ApproximatePercentile approximatePercentile) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + } + @Override public void visit(ColumnAggregation columnAgg) { out = columnAgg.spec() diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 842188e4696..8a14f4077d9 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -11,6 +11,9 @@ * * @see io.deephaven.api.TableOperations#aggBy(Collection, Collection) * @see Count + * @see FirstRowKey + * @see LastRowKey + * @see ApproximatePercentile * @see ColumnAggregation * @see ColumnAggregations */ @@ -160,6 +163,8 @@ interface Visitor { void visit(LastRowKey lastRowKey); + void visit(ApproximatePercentile approximatePercentile); + void visit(ColumnAggregation columnAgg); void visit(ColumnAggregations columnAggs); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java index de7ee008dbe..5dc8ce01c34 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java @@ -7,7 +7,7 @@ /** * A visitor to describe the input and aggregation {@link Pair column name pairs} for {@link Aggregation aggregations}. */ -public class AggregationDescriptions implements Aggregation.Visitor { +public final class AggregationDescriptions implements Aggregation.Visitor { public static Map of(Aggregation aggregation) { return aggregation.walk(new AggregationDescriptions()).getOut(); @@ -40,6 +40,13 @@ public void visit(LastRowKey lastRowKey) { out.put(lastRowKey.column().name(), "last row key"); } + @Override + public void visit(ApproximatePercentile approximatePercentile) { + approximatePercentile.percentileOutputs().forEach(po -> out.put(po.output().name(), String.format( + "%s aggregated with %.2f approximate percentile (compression %.2f)", + approximatePercentile.input().name(), po.percentile(), approximatePercentile.compression()))); + } + @Override public void visit(ColumnAggregation columnAgg) { visitColumnAgg(columnAgg.pair(), columnAgg.spec().description()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java index 0fa72923d35..7da29dbfa68 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java @@ -50,6 +50,8 @@ public List build() { for (Pair pair : e.getValue()) { out.add(LastRowKey.of((ColumnName) pair)); } + } else if (e.getValue() == null) { + out.add((Aggregation) e.getKey()); } else if (e.getValue().size() == 1) { out.add(ColumnAggregation.of((AggSpec) e.getKey(), e.getValue().get(0))); } else { @@ -74,6 +76,11 @@ public void visit(LastRowKey lastRowKey) { visitOrder.computeIfAbsent(LAST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(lastRowKey.column()); } + @Override + public void visit(ApproximatePercentile approximatePercentile) { + visitOrder.computeIfAbsent(approximatePercentile, k -> null); + } + @Override public void visit(ColumnAggregation columnAgg) { visitOrder.computeIfAbsent(columnAgg.spec(), k -> new ArrayList<>()).add(columnAgg.pair()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java index d5838dff42b..102bec7f663 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java @@ -48,6 +48,17 @@ public void visit(LastRowKey lastRowKey) { out = Stream.of(lastRowKey.column()); } + @Override + public void visit(ApproximatePercentile approximatePercentile) { + final Stream percentilePairs = approximatePercentile.percentileOutputs().stream() + .map(po -> Pair.of(approximatePercentile.input(), po.output())); + if (approximatePercentile.digest().isPresent()) { + out = Stream.concat(Stream.of(approximatePercentile.digest().get()), percentilePairs); + } else { + out = percentilePairs; + } + } + @Override public void visit(ColumnAggregation columnAgg) { out = Stream.of(columnAgg.pair()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java b/table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java new file mode 100644 index 00000000000..b642e5ad0a5 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java @@ -0,0 +1,125 @@ +package io.deephaven.api.agg; + +import io.deephaven.annotations.BuildableStyle; +import io.deephaven.annotations.TupleStyle; +import io.deephaven.api.ColumnName; +import org.immutables.value.Value.Check; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; + +import java.util.List; +import java.util.Optional; + +/** + * Approximate percentile aggregation using a T-Digest for calculation. Efficiently supports multiple output percentiles + * based on a single input column. May only be used on static or add-only tables. + */ +@Immutable +@BuildableStyle +public abstract class ApproximatePercentile implements Aggregation { + + public static ApproximatePercentile.Builder builder() { + return ImmutableApproximatePercentile.builder(); + } + + @Immutable + @TupleStyle + public static abstract class PercentileOutput { + + public static PercentileOutput of(double percentile, ColumnName output) { + return PercentileOutputTuple.of(percentile, output); + } + + /** + * Percentile. Must be in range [0.0, 1.0]. + * + * @return The percentile + */ + public abstract double percentile(); + + /** + * Output {@link ColumnName column name}. + * + * @return The output {@link ColumnName column name} + */ + public abstract ColumnName output(); + + @Check + final void checkPercentile() { + if (percentile() < 0.0 || percentile() > 1.0) { + throw new IllegalArgumentException("Percentile must be in range [0.0, 1.0]"); + } + } + } + + /** + * Input {@link ColumnName column name}. + * + * @return The input {@link ColumnName column name} + */ + public abstract ColumnName input(); + + /** + * T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large. + * + * @return The T-Digest compression factor + */ + @Default + public double compression() { + return 100.0; + } + + /** + * Optional {@link ColumnName column name} to expose the T-Digest used in approximate percentile calculation. + * + * @return The optional T-Digest {@link ColumnName column name} + */ + public abstract Optional digest(); + + /** + * Pairs linking desired approximate percentiles with their output {@link ColumnName column name}. + * + * @return The percentile-output pairs + */ + public abstract List percentileOutputs(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } + + @Check + final void checkCompression() { + if (compression() < 1.0) { + throw new IllegalArgumentException("Compression must be in greater than or equal to 1.0"); + } + } + + @Check + final void checkNonEmpty() { + if (percentileOutputs().isEmpty()) { + throw new IllegalArgumentException("ApproximatePercentile should have at least one percentile output"); + } + } + + public interface Builder { + ApproximatePercentile.Builder input(ColumnName input); + + ApproximatePercentile.Builder compression(double compression); + + ApproximatePercentile.Builder digest(ColumnName digest); + + default ApproximatePercentile.Builder addPercentileOutput(double percentile, ColumnName output) { + return addPercentileOutputs(PercentileOutput.of(percentile, output)); + } + + ApproximatePercentile.Builder addPercentileOutputs(PercentileOutput percentileOutputs); + + ApproximatePercentile.Builder addPercentileOutputs(PercentileOutput... percentileOutputs); + + ApproximatePercentile.Builder addAllPercentileOutputs(Iterable elements); + + ApproximatePercentile build(); + } +} From 29eca75f64f46a492c63e01ac90bb8221541c817 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 28 Jan 2022 19:40:38 -0500 Subject: [PATCH 21/44] Finished refactor of ApproximatePercentile --- .../engine/PercentileByBenchmark.java | 10 +- .../table/impl/by/AggregationContext.java | 2 +- .../impl/by/AggregationElementAdapter.java | 6 - .../table/impl/by/AggregationProcessor.java | 107 +++++--- .../table/impl/by/ApproximatePercentile.java | 231 +++--------------- .../by/ChunkedOperatorAggregationHelper.java | 1 + .../IterativeChunkedAggregationOperator.java | 2 +- .../impl/by/TDigestPercentileOperator.java | 217 +++++++++++++--- .../table/impl/QueryTableAggregationTest.java | 53 ++-- .../client/impl/BatchTableRequestBuilder.java | 40 +-- .../io/deephaven/api/agg/Aggregation.java | 95 ++++++- .../api/agg/AggregationDescriptions.java | 33 +-- .../api/agg/AggregationOptimizer.java | 27 +- .../deephaven/api/agg/AggregationPairs.java | 31 +-- .../io/deephaven/api/agg/Aggregations.java | 47 ++++ .../api/agg/ApproximatePercentile.java | 125 ---------- .../io/deephaven/api/agg/spec/AggSpec.java | 20 ++ .../deephaven/api/agg/spec/AggSpecAbsSum.java | 1 - .../spec/AggSpecApproximatePercentile.java | 68 ++++++ .../api/agg/spec/AggSpecColumnReferences.java | 10 + .../api/agg/spec/AggSpecTDigest.java | 56 +++++ .../api/agg/util/PercentileOutput.java | 47 ++++ 22 files changed, 698 insertions(+), 531 deletions(-) create mode 100644 table-api/src/main/java/io/deephaven/api/agg/Aggregations.java delete mode 100644 table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/util/PercentileOutput.java diff --git a/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java b/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java index db990733d3e..2d02ec1ac1b 100644 --- a/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java +++ b/engine/benchmark/src/benchmark/java/io/deephaven/benchmark/engine/PercentileByBenchmark.java @@ -9,7 +9,6 @@ import io.deephaven.engine.table.impl.select.SelectColumnFactory; import io.deephaven.engine.util.TableTools; import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.by.*; import io.deephaven.benchmarking.*; import io.deephaven.benchmarking.generator.ColumnGenerator; import io.deephaven.benchmarking.generator.EnumStringColumnGenerator; @@ -200,13 +199,8 @@ private Function getFunction() { if (percentileMode.equals("normal")) { fut = t -> t.aggAllBy(AggSpec.percentile(0.99), SelectColumn.from(Selectable.from(keyColumnNames))); } else if (percentileMode.equals("tdigest")) { - fut = (t) -> { - final NonKeyColumnAggregationFactory aggregationContextFactory = - new NonKeyColumnAggregationFactory((type, resultName, - exposeInternalColumns) -> new TDigestPercentileOperator(type, 100.0, 0.99, resultName)); - return ChunkedOperatorAggregationHelper.aggregation(aggregationContextFactory, (QueryTable) t, - SelectColumnFactory.getExpressions(keyColumnNames)); - }; + fut = (t) -> t.aggAllBy(AggSpec.approximatePercentile(0.99, 100.0), + SelectColumnFactory.getExpressions(keyColumnNames)); } else { throw new IllegalArgumentException("Bad mode: " + percentileMode); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java index 3ab42822479..c2897ed5931 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java @@ -263,7 +263,7 @@ void resetOperatorsForStep(@NotNull final TableUpdate upstream) { * keys to > 0), removed (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by * this iteration. Note that the arguments to this method should not be mutated in any way. * - * @param downstream The downstream {@link TableUpdateImpl} (which does not have its + * @param downstream The downstream {@link TableUpdate} (which does not have its * {@link ModifiedColumnSet} finalized yet) * @param newDestinations New destinations added on this update */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java index d102f5f5d02..9706f9a330b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java @@ -1,7 +1,6 @@ package io.deephaven.engine.table.impl.by; import io.deephaven.api.agg.*; -import io.deephaven.api.agg.ApproximatePercentile; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement; import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElementImpl; @@ -42,11 +41,6 @@ public void visit(LastRowKey lastRowKey) { MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY); } - @Override - public void visit(ApproximatePercentile approximatePercentile) { - // WHO CARE I'm DELETING THIS CLASS - } - @Override public void visit(ColumnAggregation columnAgg) { final AggregationSpec spec = AggregationSpecAdapter.of(columnAgg.spec()); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index d9be876c59f..a4fc2a4df83 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -3,7 +3,6 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; import io.deephaven.api.agg.*; -import io.deephaven.api.agg.ApproximatePercentile; import io.deephaven.api.agg.spec.*; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.attributes.Values; @@ -35,11 +34,9 @@ import java.util.function.Function; import java.util.function.ToIntFunction; import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.Stream; -import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; -import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY; +import static io.deephaven.datastructures.util.CollectionUtil.*; import static io.deephaven.engine.table.ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY; import static io.deephaven.engine.table.Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE; import static io.deephaven.engine.table.Table.REVERSE_LOOKUP_ATTRIBUTE; @@ -283,6 +280,37 @@ final void addBasicOperators( } } + final void addApproximatePercentileOperators(final double percentile, final double compression) { + for (final Pair pair : resultPairs) { + final String inputName = pair.input().name(); + final String resultName = pair.output().name(); + + addApproximatePercentileOperator(percentile, compression, inputName, resultName); + } + } + + final void addApproximatePercentileOperator(final double percentile, final double compression, + @NotNull final String inputName, @NotNull final String resultName) { + final ColumnSource inputSource = table.getColumnSource(inputName); + final Class type = inputSource.getType(); + + final int size = inputSources.size(); + for (int ii = 0; ii < size; ii++) { + final IterativeChunkedAggregationOperator operator; + if (inputSources.get(ii) == inputSource && + (operator = operators.get(ii)) instanceof TDigestPercentileOperator) { + final TDigestPercentileOperator tDigestOperator = (TDigestPercentileOperator) operator; + if (tDigestOperator.compression() == compression) { + addOperator(tDigestOperator.makeSecondaryOperator(percentile, resultName), inputSource, + inputName); + return; + } + } + } + addOperator(new TDigestPercentileOperator(type, compression, percentile, resultName), inputSource, + inputName); + } + final void addMinOrMaxOperators(final boolean isMin) { for (final Pair pair : resultPairs) { final String inputName = pair.input().name(); @@ -298,20 +326,19 @@ final void addMinOrMaxOperator(final boolean isMin, @NotNull final String inputN final Class type = rawInputSource.getType(); final ColumnSource inputSource = maybeReinterpretDateTimeAsLong(rawInputSource); - IntStream.range(0, inputSources.size()) - .filter(index -> (inputSources.get(index) == inputSource) - && (operators.get(index) instanceof SsmChunkedMinMaxOperator)) - .findFirst().ifPresentOrElse( - (final int priorMinMaxIndex) -> { - final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = - (SsmChunkedMinMaxOperator) operators.get(priorMinMaxIndex); - addOperator( - ssmChunkedMinMaxOperator.makeSecondaryOperator(isMin, resultName), - null, inputName); - }, - () -> addOperator( - makeMinOrMaxOperator(type, resultName, isMin, isAddOnly || isStream), - inputSource, inputName)); + final int size = inputSources.size(); + for (int ii = 0; ii < size; ii++) { + if (inputSources.get(ii) != inputSource) { + continue; + } + final IterativeChunkedAggregationOperator operator = operators.get(ii); + if (operator instanceof SsmChunkedMinMaxOperator) { + final SsmChunkedMinMaxOperator minMaxOperator = (SsmChunkedMinMaxOperator) operator; + addOperator(minMaxOperator.makeSecondaryOperator(isMin, resultName), null, inputName); + return; + } + } + addOperator(makeMinOrMaxOperator(type, resultName, isMin, isAddOnly || isStream), inputSource, inputName); } final void addFirstOrLastOperators(final boolean isFirst, final String exposeRedirectionAs) { @@ -481,21 +508,6 @@ public void visit(@NotNull final LastRowKey lastRowKey) { addFirstOrLastOperators(false, lastRowKey.column().name()); } - @Override - public void visit(ApproximatePercentile approximatePercentile) { - final String inputName = approximatePercentile.input().name(); - final ColumnSource inputSource = table.getColumnSource(inputName); - final IterativeChunkedAggregationOperator operator = new TDigestPercentileOperator( - inputSource.getType(), - approximatePercentile.compression(), - approximatePercentile.digest().map(ColumnName::name).orElse(null), - approximatePercentile.percentileOutputs().stream() - .mapToDouble(ApproximatePercentile.PercentileOutput::percentile).toArray(), - approximatePercentile.percentileOutputs().stream() - .map(po -> po.output().name()).toArray(String[]::new)); - addOperator(operator, inputSource, inputName); - } - // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -505,6 +517,11 @@ public void visit(@NotNull final AggSpecAbsSum absSum) { addBasicOperators((t, n) -> makeSumOperator(t, n, true)); } + @Override + public void visit(@NotNull final AggSpecApproximatePercentile pct) { + addApproximatePercentileOperators(pct.percentile(), pct.compression()); + } + @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { addBasicOperators((t, n) -> makeCountDistinctOperator(t, n, countDistinct.countNulls(), false, false)); @@ -586,6 +603,11 @@ public void visit(@NotNull final AggSpecSum sum) { addBasicOperators((t, n) -> makeSumOperator(t, n, false)); } + public void visit(@NotNull final AggSpecTDigest tDigest) { + addBasicOperators((t, n) -> new TDigestPercentileOperator(t, tDigest.compression(), n, + ZERO_LENGTH_DOUBLE_ARRAY, ZERO_LENGTH_STRING_ARRAY)); + } + @Override public void visit(@NotNull final AggSpecUnique unique) { addBasicOperators((t, n) -> makeUniqueOperator(t, n, @@ -630,15 +652,16 @@ default void visit(@NotNull final LastRowKey lastRowKey) { rollupUnsupported("LastRowKey"); } - @Override - default void visit(ApproximatePercentile approximatePercentile) { - rollupUnsupported("ApproximatePercentile"); - } - // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor for unsupported column aggregation specs // ------------------------------------------------------------------------------------------------------------- + @Override + @FinalDefault + default void visit(@NotNull final AggSpecApproximatePercentile approxPct) { + rollupUnsupported("ApproximatePercentile"); + } + @Override @FinalDefault default void visit(@NotNull final AggSpecGroup group) { @@ -663,6 +686,12 @@ default void visit(@NotNull final AggSpecPercentile pct) { rollupUnsupported("Percentile"); } + @Override + @FinalDefault + default void visit(@NotNull final AggSpecTDigest tDigest) { + rollupUnsupported("TDigest"); + } + @Override @FinalDefault default void visit(@NotNull final AggSpecWAvg wAvg) { @@ -1155,7 +1184,7 @@ private static class WeightedOpResult { private final ColumnSource source; private WeightedOpResult(@NotNull final Pair pair, @NotNull final WeightedOpResultType type, - @NotNull final ColumnSource source) { + @NotNull final ColumnSource source) { this.pair = pair; this.type = type; this.source = source; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java index 37d14037cc4..8940252325b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ApproximatePercentile.java @@ -1,18 +1,11 @@ package io.deephaven.engine.table.impl.by; -import io.deephaven.configuration.Configuration; -import io.deephaven.datastructures.util.CollectionUtil; +import com.tdunning.math.stats.TDigest; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; import io.deephaven.engine.table.Table; -import io.deephaven.vector.ObjectVector; -import io.deephaven.engine.table.impl.select.SelectColumnFactory; -import io.deephaven.api.util.NameValidator; -import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.table.impl.select.SelectColumn; -import com.tdunning.math.stats.TDigest; -import gnu.trove.list.array.TDoubleArrayList; - -import java.util.ArrayList; -import java.util.List; +import io.deephaven.vector.ObjectVector; /** * Generate approximate percentile aggregations of a table. @@ -30,8 +23,7 @@ * *

* The input table must be add only, if modifications or removals take place; then an UnsupportedOperationException is - * thrown. For tables with adds and removals you must use exact percentiles with - * {@link AggregationFactory#AggPct(double, java.lang.String...)}. + * thrown. For tables with adds and removals you must use exact percentiles with {@link Aggregation#AggPct}. *

* *

@@ -49,25 +41,22 @@ * and 99th percentiles of the "Size" column by "Sym": * *

- * new ApproximatePercentile.PercentileDefinition("Latency").add(0.75, "L75").add(0.95, "L95").add(0.99, "L99")
- *         .nextColumn("Size").add(0.95, "S95").add(0.99, "S99");
- * final Table aggregated = ApproximatePercentile.approximatePercentiles(input, definition);
+ * final Table aggregated = input.aggBy(List.of(
+ *         Aggregation.ApproxPct("Latency', PctOut(0.75, "L75"), PctOut(0.95, "L95"), PctOut(0.99, "L99")
+ *         Aggregation.ApproxPct("Size', PctOut(0.95, "S95"), PctOut(0.99, "S99")));
  * 
*

* *

* When parallelizing a workload, you may want to divide it based on natural partitioning and then compute an overall - * percentile. In these cases, you should use the {@link PercentileDefinition#exposeDigest} method to expose the - * internal t-digest structure as a column. If you then perform an array aggregation ({@link Table#groupBy}), you can - * call the {@link #accumulateDigests} function to produce a single digest that represents all of the constituent - * digests. The amount of error introduced is related to the compression factor that you have selected for the digests. - * Once you have a combined digest object, you can call the quantile or other functions to extract the desired - * percentile. + * percentile. In these cases, you should use the {@link Aggregation#AggTDigest} aggregation to expose the internal + * t-digest structure as a column. If you then perform an array aggregation ({@link Table#groupBy}), you can call the + * {@link #accumulateDigests} function to produce a single digest that represents all of the constituent digests. The + * amount of error introduced is related to the compression factor that you have selected for the digests. Once you have + * a combined digest object, you can call the quantile or other functions to extract the desired percentile. *

*/ public class ApproximatePercentile { - public static double DEFAULT_COMPRESSION = - Configuration.getInstance().getDoubleWithDefault("ApproximatePercentile.defaultCompression", 100.0); // static usage only private ApproximatePercentile() {} @@ -80,9 +69,8 @@ private ApproximatePercentile() {} * @return a single row table with double columns representing the approximate percentile for each column of the * input table */ - public static Table approximatePercentile(Table input, double percentile) { - return approximatePercentile(input, DEFAULT_COMPRESSION, percentile, - SelectColumn.ZERO_LENGTH_SELECT_COLUMN_ARRAY); + public static Table approximatePercentileBy(Table input, double percentile) { + return input.aggAllBy(AggSpecApproximatePercentile.of(percentile)); } /** @@ -94,9 +82,8 @@ public static Table approximatePercentile(Table input, double percentile) { * @return a with the groupByColumns and double columns representing the approximate percentile for each remaining * column of the input table */ - public static Table approximatePercentile(Table input, double percentile, String... groupByColumns) { - return approximatePercentile(input, DEFAULT_COMPRESSION, percentile, - SelectColumnFactory.getExpressions(groupByColumns)); + public static Table approximatePercentileBy(Table input, double percentile, String... groupByColumns) { + return input.aggAllBy(AggSpecApproximatePercentile.of(percentile), groupByColumns); } /** @@ -108,8 +95,8 @@ public static Table approximatePercentile(Table input, double percentile, String * @return a with the groupByColumns and double columns representing the approximate percentile for each remaining * column of the input table */ - public static Table approximatePercentile(Table input, double percentile, SelectColumn... groupByColumns) { - return approximatePercentile(input, DEFAULT_COMPRESSION, percentile, groupByColumns); + public static Table approximatePercentileBy(Table input, double percentile, SelectColumn... groupByColumns) { + return input.aggAllBy(AggSpecApproximatePercentile.of(percentile), groupByColumns); } /** @@ -122,176 +109,13 @@ public static Table approximatePercentile(Table input, double percentile, Select * @return a with the groupByColumns and double columns representing the approximate percentile for each remaining * column of the input table */ - public static Table approximatePercentile(Table input, double compression, double percentile, + public static Table approximatePercentileBy(Table input, double compression, double percentile, SelectColumn... groupByColumns) { - final NonKeyColumnAggregationFactory aggregationContextFactory = new NonKeyColumnAggregationFactory( - (type, resultName, exposeInternalColumns) -> new TDigestPercentileOperator(type, compression, - percentile, resultName)); - return ChunkedOperatorAggregationHelper.aggregation(aggregationContextFactory, (QueryTable) input, - groupByColumns); - } - - /** - * A builder class for an approximate percentile definition to be used with {@link #approximatePercentiles}. - */ - public static class PercentileDefinition { - private final static PercentileDefinition[] ZERO_LENGTH_PERCENTILE_DEFINITION_ARRAY = - new PercentileDefinition[0]; - - private final PercentileDefinition prior; - private final PercentileDefinition first; - - private final String inputColumn; - private String digestColumnName; - private final TDoubleArrayList percentiles; - private final List resultNames; - - double compression = DEFAULT_COMPRESSION; - - /** - * Create a builder with the current input column set to inputColumn. - * - * @param inputColumn the current input column - */ - public PercentileDefinition(String inputColumn) { - this(inputColumn, null); - } - - private PercentileDefinition(String inputColumn, PercentileDefinition prior) { - this.inputColumn = inputColumn; - percentiles = new TDoubleArrayList(); - resultNames = new ArrayList<>(); - digestColumnName = null; - this.prior = prior; - this.first = prior == null ? this : prior.first; - } - - /** - * Adds an output column. - * - * To set the inputColumn call {@link #nextColumn(String)}. - * - * @param percentile the percentile to calculate - * @param resultName the result name - * - * @return a (possibly new) PercentileDefinition - */ - public PercentileDefinition add(double percentile, String resultName) { - percentiles.add(percentile); - resultNames.add(NameValidator.validateColumnName(resultName)); - return this; - } - - /** - * Sets the name of the inputColumn - * - * @param inputColumn the name of the input column that subsequent calls to {@link #add} operate on. - * - * @return a (possibly new) PercentileDefinition - */ - public PercentileDefinition nextColumn(String inputColumn) { - return new PercentileDefinition(inputColumn, this); - } - - /** - * Sets the t-digest compression parameter. - * - * @param compression the t-digest compression factor. - * - * @return a (possibly new) PercentileDefinition - */ - public PercentileDefinition setCompression(double compression) { - first.compression = compression; - return this; - } - - /** - * If true, the tDigest column is exposed using the given name - * - * @param digestColumnName the name of the t-digest column in the output - * - * @return a (possibly new) PercentileDefinition - */ - public PercentileDefinition exposeDigest(String digestColumnName) { - this.digestColumnName = digestColumnName; - return this; - } - - private static List flatten(PercentileDefinition value) { - final List result = new ArrayList<>(); - value.flattenInto(result); - return result; - } - - private void flattenInto(List result) { - if (prior != null) { - prior.flattenInto(result); - } - result.add(this); - } - } - - /** - * Compute a set of approximate percentiles for input according to the definitions in percentileDefinitions. - * - * @param input the table to compute approximate percentiles for - * @param percentileDefinitions the compression factor, and map of input columns to output columns - * @param groupByColumns the columns to group by - * @return a table containing the groupByColumns and the approximate percentiles - */ - public static Table approximatePercentiles(Table input, PercentileDefinition percentileDefinitions, - SelectColumn... groupByColumns) { - final List flatDefs = PercentileDefinition.flatten(percentileDefinitions); - if (flatDefs.isEmpty()) { - throw new IllegalArgumentException("No percentile columns defined!"); - } - final double compression = flatDefs.get(0).compression; - - io.deephaven.api.agg.ApproximatePercentile.Builder aggBuilder = - io.deephaven.api.agg.ApproximatePercentile.builder(); - aggBuilder.input(flatDefs) - - final NonKeyColumnAggregationFactory aggregationContextFactory = - new NonKeyColumnAggregationFactory((type, resultName, exposeInternalColumns) -> { - for (final PercentileDefinition percentileDefinition : flatDefs) { - if (percentileDefinition.inputColumn.equals(resultName)) { - return new TDigestPercentileOperator(type, compression, - percentileDefinition.digestColumnName, percentileDefinition.percentiles.toArray(), - percentileDefinition.resultNames.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); - } - } - return null; - }); - return ChunkedOperatorAggregationHelper.aggregation(aggregationContextFactory, (QueryTable) input, - groupByColumns); - } - - /** - * Compute a set of approximate percentiles for input according to the definitions in percentileDefinitions. - * - * @param input the table to compute approximate percentiles for - * @param percentileDefinitions the compression factor, and map of input columns to output columns - * @param groupByColumns the columns to group by - * @return a table containing the groupByColumns and the approximate percentiles - */ - public static Table approximatePercentiles(Table input, PercentileDefinition percentileDefinitions, - String... groupByColumns) { - return approximatePercentiles(input, percentileDefinitions, SelectColumnFactory.getExpressions(groupByColumns)); - } - - /** - * Compute a set of approximate percentiles for input according to the definitions in percentileDefinitions. - * - * @param input the table to compute approximate percentiles for - * @param percentileDefinitions the compression factor, and map of input columns to output columns - * @return a table containing a single row with the the approximate percentiles - */ - public static Table approximatePercentiles(Table input, PercentileDefinition percentileDefinitions) { - return approximatePercentiles(input, percentileDefinitions, SelectColumn.ZERO_LENGTH_SELECT_COLUMN_ARRAY); + return input.aggAllBy(AggSpecApproximatePercentile.of(percentile, compression), groupByColumns); } /** - * Accumulate an Vector of TDigests into a single new TDigest. + * Accumulate a Vector of TDigests into a single new TDigest. * *

* Accumulate the digests within the Vector into a single TDigest. The compression factor is one third of the @@ -300,11 +124,12 @@ public static Table approximatePercentiles(Table input, PercentileDefinition per *

* *

- * This function is intended to be used for parallelization. The first step is to independently compute approximate - * percentiles with an exposed digest column using your desired buckets. Next, call {@link Table#groupBy(String...)} - * to produce arrays of Digests for each relevant bucket. Once the arrays are created, use this function to - * accumulate the arrays of digests within an {@link Table#update(String...)} statement. Finally, you may call the - * TDigest quantile function (or others) to produce the desired approximate percentile. + * This function is intended to be used for parallelization. The first step is to independently expose a T-Digest + * aggregation column with the appropriate compression factor on each of a set of sub-tables, using + * {@link Aggregation#AggTDigest} and {@link Table#aggBy}. Next, call {@link Table#groupBy(String...)} to produce + * arrays of Digests for each relevant bucket. Once the arrays are created, use this function to accumulate the + * arrays of digests within an {@link Table#update} statement. Finally, you may call the TDigest quantile function + * (or others) to produce the desired approximate percentile. *

* * @param array an array of TDigests diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java index 21648d342c8..15bde5af758 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ChunkedOperatorAggregationHelper.java @@ -6,6 +6,7 @@ import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; import io.deephaven.chunk.attributes.Values; +import io.deephaven.configuration.Configuration; import io.deephaven.datastructures.util.SmartKey; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.*; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java index 0a77c6a39af..56f4174f372 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java @@ -312,7 +312,7 @@ default void resetForStep(@NotNull final TableUpdate upstream) {} * (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that * the arguments to this method should not be mutated in any way. * - * @param downstream The downstream ShiftAwareListener.Update (which does not have its + * @param downstream The downstream TableUpdate (which does not have its * {@link ModifiedColumnSet} finalized yet) * @param newDestinations New destinations added on this update */ diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java index c30dba5d56d..9fce8fad764 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java @@ -4,37 +4,47 @@ package io.deephaven.engine.table.impl.by; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.time.DateTime; -import io.deephaven.util.QueryConstants; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.sources.*; +import com.tdunning.math.stats.TDigest; +import io.deephaven.base.verify.Assert; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.ChunkLengths; import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.*; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.sources.DoubleArraySource; +import io.deephaven.engine.table.impl.sources.ObjectArraySource; import io.deephaven.engine.table.impl.util.cast.ToDoubleCast; -import com.tdunning.math.stats.TDigest; +import io.deephaven.time.DateTime; +import io.deephaven.util.QueryConstants; import org.jetbrains.annotations.NotNull; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; import java.util.function.Supplier; /** - * Iterative average operator. + * Iterative T-Digest and percentile operator. */ public class TDigestPercentileOperator implements IterativeChunkedAggregationOperator { - private final DoubleArraySource[] resultColumns; + + private final double compression; private final double[] percentiles; - private final ObjectArraySource digests; + private final String digestColumnName; private final String[] resultNames; - private final Supplier digestFactory; + private final ChunkType chunkType; - private final String digestColumnName; + private final ObjectArraySource digests; + private final DoubleArraySource[] resultColumns; + private final Supplier digestFactory; + + private long firstNewDestination; + private boolean modifiedThisStep; + private WritableIntChunk chunkModifiedPositions; public TDigestPercentileOperator(@NotNull Class type, double compression, double percentile, @NotNull String name) { @@ -44,9 +54,14 @@ public TDigestPercentileOperator(@NotNull Class type, double compression, dou public TDigestPercentileOperator(@NotNull Class type, double compression, String digestColumnName, @NotNull double[] percentiles, @NotNull String[] resultNames) { if (resultNames.length != percentiles.length) { - throw new IllegalArgumentException("Percentile length and resultName length must be identical:" + throw new IllegalArgumentException("percentiles length and resultName length must be identical:" + resultNames.length + " (resultNames) != " + percentiles.length + " (percentiles)"); } + if (resultNames.length == 0 && digestColumnName == null) { + throw new IllegalArgumentException( + "Must have at least one result column; must have at least one percentile result or provide a digest column name to expose"); + } + this.compression = compression; this.percentiles = percentiles; this.digestColumnName = digestColumnName; this.resultNames = resultNames; @@ -63,6 +78,18 @@ public TDigestPercentileOperator(@NotNull Class type, double compression, Str digestFactory = () -> TDigest.createDigest(compression); } + public double compression() { + return compression; + } + + private static UnsupportedOperationException modificationUnsupported() { + return new UnsupportedOperationException("t-digest Approximate percentiles do not support data modification"); + } + + private static UnsupportedOperationException removalUnsupported() { + return new UnsupportedOperationException("t-digest Approximate percentiles do not support data removal"); + } + @Override public void addChunk(BucketedContext bucketedContext, Chunk values, LongChunk inputRowKeys, IntChunk destinations, @@ -71,6 +98,7 @@ public void addChunk(BucketedContext bucketedContext, Chunk va final TDigestContext tDigestContext = (TDigestContext) bucketedContext; final DoubleChunk doubleValues = tDigestContext.toDoubleCast.cast(values); + chunkModifiedPositions.setSize(0); for (int ii = 0; ii < startPositions.size(); ++ii) { final int runLength = length.get(ii); if (runLength == 0) { @@ -91,7 +119,11 @@ public void addChunk(BucketedContext bucketedContext, Chunk va digest.add(x); } if (added > 0) { + if (destination < firstNewDestination) { + modifiedThisStep = true; + } stateModified.set(ii, true); + chunkModifiedPositions.add(ii); } } } @@ -101,7 +133,7 @@ public void removeChunk(BucketedContext bucketedContext, Chunk LongChunk inputRowKeys, IntChunk destinations, IntChunk startPositions, IntChunk length, WritableBooleanChunk stateModified) { - throw new UnsupportedOperationException("t-digest Approximate percentiles do not support data removal."); + throw removalUnsupported(); } @Override @@ -109,7 +141,7 @@ public void modifyChunk(BucketedContext bucketedContext, Chunk Chunk postValues, LongChunk postShiftRowKeys, IntChunk destinations, IntChunk startPositions, IntChunk length, WritableBooleanChunk stateModified) { - throw new UnsupportedOperationException("t-digest Approximate percentiles do not support data modification."); + throw modificationUnsupported(); } @Override @@ -119,6 +151,7 @@ public boolean addChunk(SingletonContext singletonContext, int chunkSize, Chunk< final TDigest digest = digestForSlot(destination); final DoubleChunk doubleValues = tDigestContext.toDoubleCast.cast(values); + chunkModifiedPositions.setSize(0); int added = 0; for (int ii = 0; ii < doubleValues.size(); ++ii) { final double x = doubleValues.get(ii); @@ -129,15 +162,38 @@ public boolean addChunk(SingletonContext singletonContext, int chunkSize, Chunk< digest.add(x); } - return added > 0; + if (added > 0) { + if (destination < firstNewDestination) { + modifiedThisStep = true; + } + chunkModifiedPositions.add(0); + return true; + } + return false; } @Override - public void propagateInitialState(@NotNull QueryTable resultTable) { + public boolean removeChunk(SingletonContext singletonContext, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + throw removalUnsupported(); + } + + @Override + public boolean modifyChunk(SingletonContext context, int chunkSize, Chunk previousValues, + Chunk newValues, LongChunk postShiftRowKeys, long destination) { + throw modificationUnsupported(); + } + + @Override + public void propagateInitialState(@NotNull final QueryTable resultTable) { + if (resultColumns.length == 0) { + return; + } + firstNewDestination = resultTable.getRowSet().lastRowKey(); resultTable.getRowSet().forAllRowKeys(this::updateDestination); } - private void updateDestination(long destination) { + private void updateDestination(final long destination) { final TDigest digest = digestForSlot(destination); for (int jj = 0; jj < resultColumns.length; ++jj) { resultColumns[jj].set(destination, digest.quantile(percentiles[jj])); @@ -145,19 +201,23 @@ private void updateDestination(long destination) { } @Override - public void propagateUpdates(@NotNull TableUpdate downstream, - @NotNull RowSet newDestinations) { - downstream.added().forAllRowKeys(this::updateDestination); - downstream.modified().forAllRowKeys(this::updateDestination); + public void resetForStep(@NotNull final TableUpdate upstream) { + modifiedThisStep = false; } @Override - public boolean removeChunk(SingletonContext singletonContext, int chunkSize, Chunk values, - LongChunk inputRowKeys, long destination) { - throw new UnsupportedOperationException("t-digest Approximate percentiles do not support data removal."); + public void propagateUpdates(@NotNull final TableUpdate downstream, @NotNull final RowSet newDestinations) { + if (resultColumns.length == 0) { + return; + } + firstNewDestination = newDestinations.lastRowKey(); + downstream.added().forAllRowKeys(this::updateDestination); + if (modifiedThisStep) { + downstream.modified().forAllRowKeys(this::updateDestination); + } } - private TDigest digestForSlot(long slot) { + private TDigest digestForSlot(final long slot) { TDigest digest = digests.getUnsafe(slot); if (digest == null) { digests.set(slot, digest = digestFactory.get()); @@ -197,11 +257,15 @@ private class TDigestContext implements SingletonContext, BucketedContext { private TDigestContext(int size) { toDoubleCast = ToDoubleCast.makeToDoubleCast(chunkType, size); + Assert.eqNull(chunkModifiedPositions, "chunkModifiedPositions"); + chunkModifiedPositions = WritableIntChunk.makeWritableChunk(size); } @Override public void close() { toDoubleCast.close(); + chunkModifiedPositions.close(); + chunkModifiedPositions = null; } } @@ -214,4 +278,101 @@ public SingletonContext makeSingletonContext(int size) { public BucketedContext makeBucketedContext(int size) { return new TDigestContext(size); } + + public IterativeChunkedAggregationOperator makeSecondaryOperator(double percentile, @NotNull String resultName) { + return new SecondaryOperator(percentile, resultName); + } + + private class SecondaryOperator implements IterativeChunkedAggregationOperator { + + private final double percentile; + private final String resultName; + + private final DoubleArraySource resultColumn; + + public SecondaryOperator(final double percentile, @NotNull final String resultName) { + this.percentile = percentile; + this.resultName = resultName; + + resultColumn = new DoubleArraySource(); + } + + + @Override + public void addChunk(BucketedContext context, Chunk values, + LongChunk inputRowKeys, IntChunk destinations, + IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) { + final int numModifiedPositions = chunkModifiedPositions.size(); + for (int mpi = 0; mpi < numModifiedPositions; ++mpi) { + stateModified.set(chunkModifiedPositions.get(mpi), true); + } + } + + @Override + public void removeChunk(BucketedContext context, Chunk values, + LongChunk inputRowKeys, IntChunk destinations, + IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) { + throw removalUnsupported(); + } + + @Override + public void modifyChunk(BucketedContext context, Chunk previousValues, + Chunk newValues, LongChunk postShiftRowKeys, + IntChunk destinations, IntChunk startPositions, IntChunk length, + WritableBooleanChunk stateModified) { + throw modificationUnsupported(); + } + + @Override + public boolean addChunk(SingletonContext context, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + return chunkModifiedPositions.size() > 0; + } + + @Override + public boolean removeChunk(SingletonContext context, int chunkSize, Chunk values, + LongChunk inputRowKeys, long destination) { + throw removalUnsupported(); + } + + @Override + public boolean modifyChunk(SingletonContext context, int chunkSize, Chunk previousValues, + Chunk newValues, LongChunk postShiftRowKeys, long destination) { + throw modificationUnsupported(); + } + + @Override + public void ensureCapacity(final long tableSize) { + resultColumn.ensureCapacity(tableSize); + } + + @Override + public Map> getResultColumns() { + return Collections.singletonMap(resultName, resultColumn); + } + + @Override + public void propagateInitialState(@NotNull final QueryTable resultTable) { + resultTable.getRowSet().forAllRowKeys(this::updateDestination); + } + + @Override + public void startTrackingPrevValues() { + resultColumn.startTrackingPrevValues(); + } + + @Override + public void propagateUpdates(@NotNull final TableUpdate downstream, @NotNull final RowSet newDestinations) { + downstream.added().forAllRowKeys(this::updateDestination); + if (modifiedThisStep) { + downstream.modified().forAllRowKeys(this::updateDestination); + } + } + + private void updateDestination(final long destination) { + resultColumn.set(destination, digestForSlot(destination).quantile(percentile)); + } + } } diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 16afa5586fa..2db57737ccc 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -1,6 +1,7 @@ package io.deephaven.engine.table.impl; import io.deephaven.api.Selectable; +import io.deephaven.api.agg.Aggregation; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; @@ -51,6 +52,7 @@ import java.util.stream.Stream; import org.junit.experimental.categories.Category; +import static io.deephaven.api.agg.Aggregation.*; import static io.deephaven.api.agg.spec.AggSpec.percentile; import static io.deephaven.engine.util.TableTools.*; import static io.deephaven.engine.table.impl.TstUtils.*; @@ -2453,10 +2455,10 @@ public void testTDigest() { new DoubleGenerator(-10000, 10000, 0.05, 0.05), new FloatGenerator(0, 100.0f))); - final Table aggregated = ApproximatePercentile.approximatePercentile(queryTable.dropColumns("Sym"), 0.99); + final Table aggregated = ApproximatePercentile.approximatePercentileBy(queryTable.dropColumns("Sym"), 0.99); TableTools.showWithRowSet(aggregated); - final Table aggregatedBySym = ApproximatePercentile.approximatePercentile(queryTable, 0.99, "Sym"); + final Table aggregatedBySym = ApproximatePercentile.approximatePercentileBy(queryTable, 0.99, "Sym"); TableTools.showWithRowSet(aggregatedBySym); checkTableP99(queryTable, aggregated); @@ -2476,15 +2478,15 @@ public void testTDigestMulti() { new DoubleGenerator(-10000, 10000, 0.05, 0.05), new FloatGenerator(0, 100.0f))); - final ApproximatePercentile.PercentileDefinition definition = - new ApproximatePercentile.PercentileDefinition("doubleCol").add(0.75, "DP75").add(0.95, "DP95") - .add(0.99, "DP99").add(0.999, "DP999").nextColumn("floatCol").add(0.75, "FP75") - .add(0.99, "FP99"); - final Table aggregated = - ApproximatePercentile.approximatePercentiles(queryTable.dropColumns("Sym"), definition); + final Collection aggregations = List.of( + AggApproxPct("doubleCol", PctOut(0.75, "DP75"), PctOut(0.95, "DP95"), PctOut(0.99, "DP99"), + PctOut(0.999, "DP999")), + AggApproxPct("floatCol", PctOut(0.75, "FP75"), PctOut(0.99, "FP99")) + ); + final Table aggregated = queryTable.dropColumns("Sym").aggBy(aggregations); TableTools.showWithRowSet(aggregated); - final Table aggregatedBySym = ApproximatePercentile.approximatePercentiles(queryTable, definition, "Sym"); + final Table aggregatedBySym = queryTable.aggBy(aggregations, "Sym"); TableTools.showWithRowSet(aggregatedBySym); checkTableComboPercentiles(queryTable, aggregated); @@ -2505,19 +2507,22 @@ public void testTDigestAccumulation() { new DoubleGenerator(-10000, 10000, 0.05, 0.05), new FloatGenerator(0, 100.0f))); - final ApproximatePercentile.PercentileDefinition definition = - new ApproximatePercentile.PercentileDefinition("doubleCol").exposeDigest("Digest").add(0.95, "P95") - .setCompression(33); - final Table aggregated = - ApproximatePercentile.approximatePercentiles(queryTable.dropColumns("Sym"), definition); + final Collection aggregations33 = List.of( + AggTDigest(33, "Digest=doubleCol"), + AggApproxPct("doubleCol", PctOut(0.95, "P95")) + ); + final Table aggregated = queryTable.dropColumns("Sym").aggBy(aggregations33); TableTools.showWithRowSet(aggregated); - final Table aggregatedBySym = - ApproximatePercentile.approximatePercentiles(queryTable, definition.setCompression(100), "Sym"); + final Collection aggregations100 = List.of( + AggTDigest(100, "Digest=doubleCol"), + AggApproxPct("doubleCol", PctOut(0.95, "P95")) + ); + final Table aggregatedBySym = queryTable.aggBy(aggregations100, "Sym"); TableTools.showWithRowSet(aggregatedBySym); final Table accumulated = aggregatedBySym.dropColumns("Sym").groupBy() - .update("Digest=io.deephaven.engine.table.impl.by.ApproximatePercentile.accumulateDigests(Digest)") + .update("Digest=io.deephaven.engine.table.impl.by.AggSpecTDigest.accumulateDigests(Digest)") .update("P95=Digest.quantile(0.95)"); TableTools.show(accumulated); @@ -2608,16 +2613,18 @@ public void testTDigestIncremental() { new DoubleGenerator(-10000, 10000, 0.05, 0.05), new LongGenerator(0, 1_000_000_000L))); - final ApproximatePercentile.PercentileDefinition definition = - new ApproximatePercentile.PercentileDefinition("doubleCol").add(0.75, "DP75").add(0.95, "DP95") - .add(0.99, "DP99").add(0.999, "DP999").nextColumn("longCol").add(0.75, "LP75").add(0.95, "LP95") - .add(0.99, "FP99").add(0.999, "LP999"); + final Collection aggregations = List.of( + AggApproxPct("doubleCol", PctOut(0.75, "DP75"), PctOut(0.95, "DP95"), PctOut(0.99, "DP99"), + PctOut(0.999, "DP999")), + AggApproxPct("longCol", PctOut(0.75, "LP75"), PctOut(0.95, "LP95"), PctOut(0.99, "LP99"), + PctOut(0.999, "LP999")) + ); final EvalNugget[] en = new EvalNugget[] { new EvalNugget() { @Override protected Table e() { - return ApproximatePercentile.approximatePercentiles(queryTable, definition); + return queryTable.aggBy(aggregations); } @Override @@ -2649,7 +2656,7 @@ void checkDifferences(String msg, Table recomputed) { new EvalNugget.Sorted(new String[] {"Sym"}) { @Override protected Table e() { - return ApproximatePercentile.approximatePercentiles(queryTable, definition, "Sym"); + return queryTable.aggBy(aggregations, "Sym"); } @Override diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index d25d814a83e..7c179857dbf 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -12,27 +12,7 @@ import io.deephaven.api.SortColumn.Order; import io.deephaven.api.Strings; import io.deephaven.api.agg.*; -import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.api.agg.spec.AggSpecAbsSum; -import io.deephaven.api.agg.spec.AggSpecAvg; -import io.deephaven.api.agg.spec.AggSpecCountDistinct; -import io.deephaven.api.agg.spec.AggSpecDistinct; -import io.deephaven.api.agg.spec.AggSpecFirst; -import io.deephaven.api.agg.spec.AggSpecFormula; -import io.deephaven.api.agg.spec.AggSpecGroup; -import io.deephaven.api.agg.spec.AggSpecLast; -import io.deephaven.api.agg.spec.AggSpecMax; -import io.deephaven.api.agg.spec.AggSpecMedian; -import io.deephaven.api.agg.spec.AggSpecMin; -import io.deephaven.api.agg.spec.AggSpecPercentile; -import io.deephaven.api.agg.spec.AggSpecSortedFirst; -import io.deephaven.api.agg.spec.AggSpecSortedLast; -import io.deephaven.api.agg.spec.AggSpecStd; -import io.deephaven.api.agg.spec.AggSpecSum; -import io.deephaven.api.agg.spec.AggSpecUnique; -import io.deephaven.api.agg.spec.AggSpecVar; -import io.deephaven.api.agg.spec.AggSpecWAvg; -import io.deephaven.api.agg.spec.AggSpecWSum; +import io.deephaven.api.agg.spec.*; import io.deephaven.api.filter.Filter; import io.deephaven.api.filter.FilterAnd; import io.deephaven.api.filter.FilterCondition; @@ -544,12 +524,6 @@ public void visit(LastRowKey lastRowKey) { "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } - @Override - public void visit(ApproximatePercentile approximatePercentile) { - throw new UnsupportedOperationException( - "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); - } - @Override public void visit(ColumnAggregation columnAgg) { out = columnAgg.spec() @@ -705,6 +679,12 @@ public void visit(AggSpecAbsSum absSum) { out = of(AggType.ABS_SUM, pairs).build(); } + @Override + public void visit(AggSpecApproximatePercentile pct) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + } + @Override public void visit(AggSpecCountDistinct countDistinct) { throw new UnsupportedOperationException( @@ -793,6 +773,12 @@ public void visit(AggSpecSum sum) { out = of(AggType.SUM, pairs).build(); } + @Override + public void visit(AggSpecTDigest tDigest) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + } + @Override public void visit(AggSpecUnique unique) { throw new UnsupportedOperationException( diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 8a14f4077d9..cce6b1f5e0f 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -1,21 +1,24 @@ package io.deephaven.api.agg; -import io.deephaven.api.agg.ColumnAggregations.Builder; +import io.deephaven.api.ColumnName; import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.api.agg.util.PercentileOutput; import java.io.Serializable; import java.util.Collection; +import java.util.function.BiFunction; /** * Represents an aggregation that can be applied to a table. * - * @see io.deephaven.api.TableOperations#aggBy(Collection, Collection) + * @see io.deephaven.api.TableOperations#aggBy + * @see io.deephaven.api.TableOperations#aggAllBy + * @see Aggregations + * @see ColumnAggregation + * @see ColumnAggregations * @see Count * @see FirstRowKey * @see LastRowKey - * @see ApproximatePercentile - * @see ColumnAggregation - * @see ColumnAggregations */ public interface Aggregation extends Serializable { @@ -27,17 +30,59 @@ static Aggregation of(AggSpec spec, String... pairs) { if (pairs.length == 1) { return of(spec, pairs[0]); } - final Builder builder = ColumnAggregations.builder().spec(spec); + final ColumnAggregations.Builder builder = ColumnAggregations.builder().spec(spec); for (String pair : pairs) { builder.addPairs(Pair.parse(pair)); } return builder.build(); } + static Aggregation of(Aggregation... aggregations) { + if (aggregations.length == 1) { + return aggregations[0]; + } + return Aggregations.builder().addAggregations(aggregations).build(); + } + + @SafeVarargs + static Aggregation of(BiFunction columnAggFactory, + String inputColumn, INPUT_TYPE... inputs) { + final ColumnName inputColumnName = ColumnName.of(inputColumn); + if (inputs.length == 1) { + return columnAggFactory.apply(inputColumnName, inputs[0]); + } + final Aggregations.Builder builder = Aggregations.builder(); + for (INPUT_TYPE input : inputs) { + builder.addAggregations(columnAggFactory.apply(inputColumnName, input)); + } + return builder.build(); + } + static Aggregation AggAbsSum(String... pairs) { return of(AggSpec.absSum(), pairs); } + static Aggregation AggApproxPct(double percentile, String... pairs) { + return of(AggSpec.approximatePercentile(percentile), pairs); + } + + static Aggregation AggApproxPct(double percentile, double compression, String... pairs) { + return of(AggSpec.approximatePercentile(percentile, compression), pairs); + } + + static Aggregation AggApproxPct(String inputColumn, PercentileOutput... percentileOutputs) { + final BiFunction aggFactory = (ic, po) -> ColumnAggregation + .of(AggSpec.approximatePercentile(po.percentile()), Pair.of(ic, po.output())); + return of(aggFactory, inputColumn, percentileOutputs); + } + + static Aggregation AggApproxPct(String inputColumn, double compression, PercentileOutput... percentileOutputs) { + final BiFunction aggFactory = + (ic, po) -> ColumnAggregation.of(AggSpec.approximatePercentile(po.percentile(), compression), + Pair.of(ic, po.output())); + return of(aggFactory, inputColumn, percentileOutputs); + } + static Aggregation AggAvg(String... pairs) { return of(AggSpec.avg(), pairs); } @@ -110,6 +155,18 @@ static Aggregation AggPct(double percentile, boolean average, String... pairs) { return of(AggSpec.percentile(percentile, average), pairs); } + static Aggregation AggPct(String inputColumn, PercentileOutput... percentileOutputs) { + final BiFunction aggFactory = + (ic, po) -> ColumnAggregation.of(AggSpec.percentile(po.percentile()), Pair.of(ic, po.output())); + return of(aggFactory, inputColumn, percentileOutputs); + } + + static Aggregation AggPct(String inputColumn, boolean average, PercentileOutput... percentileOutputs) { + final BiFunction aggFactory = (ic, po) -> ColumnAggregation + .of(AggSpec.percentile(po.percentile(), average), Pair.of(ic, po.output())); + return of(aggFactory, inputColumn, percentileOutputs); + } + static Aggregation AggSortedFirst(String sortedColumn, String... pairs) { return of(AggSpec.sortedFirst(sortedColumn), pairs); } @@ -134,6 +191,14 @@ static Aggregation AggSum(String... pairs) { return of(AggSpec.sum(), pairs); } + static Aggregation AggTDigest(String... pairs) { + return of(AggSpec.tDigest(), pairs); + } + + static Aggregation AggTDigest(double compression, String... pairs) { + return of(AggSpec.tDigest(compression), pairs); + } + static Aggregation AggUnique(String... pairs) { return of(AggSpec.unique(), pairs); } @@ -154,19 +219,25 @@ static Aggregation AggWSum(String weightColumn, String... pairs) { return of(AggSpec.wsum(weightColumn), pairs); } + static PercentileOutput PctOut(double percentile, String outputColumn) { + return PercentileOutput.of(percentile, outputColumn); + } + V walk(V visitor); interface Visitor { - void visit(Count count); + default void visit(Aggregations aggregations) { + aggregations.aggregations().forEach(a -> a.walk(this)); + } - void visit(FirstRowKey firstRowKey); + void visit(ColumnAggregation columnAgg); - void visit(LastRowKey lastRowKey); + void visit(ColumnAggregations columnAggs); - void visit(ApproximatePercentile approximatePercentile); + void visit(Count count); - void visit(ColumnAggregation columnAgg); + void visit(FirstRowKey firstRowKey); - void visit(ColumnAggregations columnAggs); + void visit(LastRowKey lastRowKey); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java index 5dc8ce01c34..b5d6d91dcf3 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java @@ -26,39 +26,32 @@ Map getOut() { } @Override - public void visit(Count count) { - out.put(count.column().name(), "count"); + public void visit(ColumnAggregation columnAgg) { + visitColumnAgg(columnAgg.pair(), columnAgg.spec().description()); } @Override - public void visit(FirstRowKey firstRowKey) { - out.put(firstRowKey.column().name(), "first row key"); + public void visit(ColumnAggregations columnAggs) { + final String specDescription = columnAggs.spec().description(); + columnAggs.pairs().forEach(p -> visitColumnAgg(p, specDescription)); } - @Override - public void visit(LastRowKey lastRowKey) { - out.put(lastRowKey.column().name(), "last row key"); + private void visitColumnAgg(Pair pair, String specDescription) { + out.put(pair.output().name(), pair.input().name() + " aggregated with " + specDescription); } @Override - public void visit(ApproximatePercentile approximatePercentile) { - approximatePercentile.percentileOutputs().forEach(po -> out.put(po.output().name(), String.format( - "%s aggregated with %.2f approximate percentile (compression %.2f)", - approximatePercentile.input().name(), po.percentile(), approximatePercentile.compression()))); + public void visit(Count count) { + out.put(count.column().name(), "count"); } @Override - public void visit(ColumnAggregation columnAgg) { - visitColumnAgg(columnAgg.pair(), columnAgg.spec().description()); + public void visit(FirstRowKey firstRowKey) { + out.put(firstRowKey.column().name(), "first row key"); } @Override - public void visit(ColumnAggregations columnAggs) { - final String specDescription = columnAggs.spec().description(); - columnAggs.pairs().forEach(p -> visitColumnAgg(p, specDescription)); - } - - private void visitColumnAgg(Pair pair, String specDescription) { - out.put(pair.output().name(), pair.input().name() + " aggregated with " + specDescription); + public void visit(LastRowKey lastRowKey) { + out.put(lastRowKey.column().name(), "last row key"); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java index 7da29dbfa68..35b5847e610 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java @@ -62,33 +62,28 @@ public List build() { } @Override - public void visit(Count count) { - visitOrder.computeIfAbsent(COUNT_OBJ, k -> new ArrayList<>()).add(count.column()); - } - - @Override - public void visit(FirstRowKey firstRowKey) { - visitOrder.computeIfAbsent(FIRST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(firstRowKey.column()); + public void visit(ColumnAggregation columnAgg) { + visitOrder.computeIfAbsent(columnAgg.spec(), k -> new ArrayList<>()).add(columnAgg.pair()); } @Override - public void visit(LastRowKey lastRowKey) { - visitOrder.computeIfAbsent(LAST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(lastRowKey.column()); + public void visit(ColumnAggregations columnAggs) { + visitOrder.computeIfAbsent(columnAggs.spec(), k -> new ArrayList<>()) + .addAll(columnAggs.pairs()); } @Override - public void visit(ApproximatePercentile approximatePercentile) { - visitOrder.computeIfAbsent(approximatePercentile, k -> null); + public void visit(Count count) { + visitOrder.computeIfAbsent(COUNT_OBJ, k -> new ArrayList<>()).add(count.column()); } @Override - public void visit(ColumnAggregation columnAgg) { - visitOrder.computeIfAbsent(columnAgg.spec(), k -> new ArrayList<>()).add(columnAgg.pair()); + public void visit(FirstRowKey firstRowKey) { + visitOrder.computeIfAbsent(FIRST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(firstRowKey.column()); } @Override - public void visit(ColumnAggregations columnAggs) { - visitOrder.computeIfAbsent(columnAggs.spec(), k -> new ArrayList<>()) - .addAll(columnAggs.pairs()); + public void visit(LastRowKey lastRowKey) { + visitOrder.computeIfAbsent(LAST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(lastRowKey.column()); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java index 102bec7f663..49b6f89fa05 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java @@ -34,38 +34,27 @@ Stream getOut() { } @Override - public void visit(Count count) { - out = Stream.of(count.column()); - } - - @Override - public void visit(FirstRowKey firstRowKey) { - out = Stream.of(firstRowKey.column()); + public void visit(ColumnAggregation columnAgg) { + out = Stream.of(columnAgg.pair()); } @Override - public void visit(LastRowKey lastRowKey) { - out = Stream.of(lastRowKey.column()); + public void visit(ColumnAggregations columnAggs) { + out = columnAggs.pairs().stream(); } @Override - public void visit(ApproximatePercentile approximatePercentile) { - final Stream percentilePairs = approximatePercentile.percentileOutputs().stream() - .map(po -> Pair.of(approximatePercentile.input(), po.output())); - if (approximatePercentile.digest().isPresent()) { - out = Stream.concat(Stream.of(approximatePercentile.digest().get()), percentilePairs); - } else { - out = percentilePairs; - } + public void visit(Count count) { + out = Stream.of(count.column()); } @Override - public void visit(ColumnAggregation columnAgg) { - out = Stream.of(columnAgg.pair()); + public void visit(FirstRowKey firstRowKey) { + out = Stream.of(firstRowKey.column()); } @Override - public void visit(ColumnAggregations columnAggs) { - out = columnAggs.pairs().stream(); + public void visit(LastRowKey lastRowKey) { + out = Stream.of(lastRowKey.column()); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java new file mode 100644 index 00000000000..b829d182612 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java @@ -0,0 +1,47 @@ +package io.deephaven.api.agg; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Check; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +import java.util.List; + +/** + * Aggregations is an {@link Aggregation} that is a collection of two or more {@link Aggregation aggregations}. + */ +@Immutable +@BuildableStyle +public abstract class Aggregations implements Aggregation { + + public static Aggregations.Builder builder() { + return ImmutableAggregations.builder(); + } + + @Parameter + public abstract List aggregations(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } + + @Check + final void checkSize() { + if (aggregations().size() < 2) { + throw new IllegalArgumentException( + String.format("%s should have at least two aggregations", Aggregations.class)); + } + } + + public interface Builder { + Builder addAggregations(Aggregation aggregation); + + Builder addAggregations(Aggregation... aggregations); + + Builder addAllAggregations(Iterable aggregations); + + Aggregations build(); + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java b/table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java deleted file mode 100644 index b642e5ad0a5..00000000000 --- a/table-api/src/main/java/io/deephaven/api/agg/ApproximatePercentile.java +++ /dev/null @@ -1,125 +0,0 @@ -package io.deephaven.api.agg; - -import io.deephaven.annotations.BuildableStyle; -import io.deephaven.annotations.TupleStyle; -import io.deephaven.api.ColumnName; -import org.immutables.value.Value.Check; -import org.immutables.value.Value.Default; -import org.immutables.value.Value.Immutable; - -import java.util.List; -import java.util.Optional; - -/** - * Approximate percentile aggregation using a T-Digest for calculation. Efficiently supports multiple output percentiles - * based on a single input column. May only be used on static or add-only tables. - */ -@Immutable -@BuildableStyle -public abstract class ApproximatePercentile implements Aggregation { - - public static ApproximatePercentile.Builder builder() { - return ImmutableApproximatePercentile.builder(); - } - - @Immutable - @TupleStyle - public static abstract class PercentileOutput { - - public static PercentileOutput of(double percentile, ColumnName output) { - return PercentileOutputTuple.of(percentile, output); - } - - /** - * Percentile. Must be in range [0.0, 1.0]. - * - * @return The percentile - */ - public abstract double percentile(); - - /** - * Output {@link ColumnName column name}. - * - * @return The output {@link ColumnName column name} - */ - public abstract ColumnName output(); - - @Check - final void checkPercentile() { - if (percentile() < 0.0 || percentile() > 1.0) { - throw new IllegalArgumentException("Percentile must be in range [0.0, 1.0]"); - } - } - } - - /** - * Input {@link ColumnName column name}. - * - * @return The input {@link ColumnName column name} - */ - public abstract ColumnName input(); - - /** - * T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large. - * - * @return The T-Digest compression factor - */ - @Default - public double compression() { - return 100.0; - } - - /** - * Optional {@link ColumnName column name} to expose the T-Digest used in approximate percentile calculation. - * - * @return The optional T-Digest {@link ColumnName column name} - */ - public abstract Optional digest(); - - /** - * Pairs linking desired approximate percentiles with their output {@link ColumnName column name}. - * - * @return The percentile-output pairs - */ - public abstract List percentileOutputs(); - - @Override - public final V walk(V visitor) { - visitor.visit(this); - return visitor; - } - - @Check - final void checkCompression() { - if (compression() < 1.0) { - throw new IllegalArgumentException("Compression must be in greater than or equal to 1.0"); - } - } - - @Check - final void checkNonEmpty() { - if (percentileOutputs().isEmpty()) { - throw new IllegalArgumentException("ApproximatePercentile should have at least one percentile output"); - } - } - - public interface Builder { - ApproximatePercentile.Builder input(ColumnName input); - - ApproximatePercentile.Builder compression(double compression); - - ApproximatePercentile.Builder digest(ColumnName digest); - - default ApproximatePercentile.Builder addPercentileOutput(double percentile, ColumnName output) { - return addPercentileOutputs(PercentileOutput.of(percentile, output)); - } - - ApproximatePercentile.Builder addPercentileOutputs(PercentileOutput percentileOutputs); - - ApproximatePercentile.Builder addPercentileOutputs(PercentileOutput... percentileOutputs); - - ApproximatePercentile.Builder addAllPercentileOutputs(Iterable elements); - - ApproximatePercentile build(); - } -} diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 824b55d17b9..fb6d62cbb95 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -18,6 +18,14 @@ static AggSpecAbsSum absSum() { return AggSpecAbsSum.of(); } + static AggSpecApproximatePercentile approximatePercentile(double percentile) { + return AggSpecApproximatePercentile.of(percentile); + } + + static AggSpecApproximatePercentile approximatePercentile(double percentile, double compression) { + return AggSpecApproximatePercentile.of(percentile, compression); + } + static AggSpecAvg avg() { return AggSpecAvg.of(); } @@ -114,6 +122,14 @@ static AggSpecSum sum() { return AggSpecSum.of(); } + static AggSpecTDigest tDigest() { + return AggSpecTDigest.of(); + } + + static AggSpecTDigest tDigest(double compression) { + return AggSpecTDigest.of(compression); + } + static AggSpecUnique unique() { return AggSpecUnique.of(); } @@ -147,6 +163,8 @@ static AggSpecWSum wsum(String weightColumn) { interface Visitor { void visit(AggSpecAbsSum absSum); + void visit(AggSpecApproximatePercentile pct); + void visit(AggSpecCountDistinct countDistinct); void visit(AggSpecDistinct distinct); @@ -177,6 +195,8 @@ interface Visitor { void visit(AggSpecSum sum); + void visit(AggSpecTDigest tDigest); + void visit(AggSpecUnique unique); void visit(AggSpecWAvg wAvg); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java index 3f3a527e974..4dbc0f4a7ab 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java @@ -1,7 +1,6 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; -import io.deephaven.api.agg.spec.ImmutableAggSpecAbsSum; import org.immutables.value.Value.Immutable; @Immutable diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java new file mode 100644 index 00000000000..11873411172 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java @@ -0,0 +1,68 @@ +package io.deephaven.api.agg.spec; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Check; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +/** + * Approximate percentile aggregation using a T-Digest for calculation. Efficiently supports multiple output percentiles + * based on a single input column. May only be used on static or add-only tables. + */ +@Immutable +@BuildableStyle +public abstract class AggSpecApproximatePercentile implements AggSpec { + + public static AggSpecApproximatePercentile of(double percentile) { + return ImmutableAggSpecApproximatePercentile.builder().percentile(percentile).build(); + } + + public static AggSpecApproximatePercentile of(double percentile, double compression) { + return ImmutableAggSpecApproximatePercentile.builder().percentile(percentile).compression(compression).build(); + } + + @Override + public final String description() { + return String.format("%.2f approximate percentile with compression %.2f", percentile(), compression()); + } + + /** + * Percentile. Must be in range [0.0, 1.0]. + * + * @return The percentile + */ + @Parameter + public abstract double percentile(); + + /** + * T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large. + * + * @return The T-Digest compression factor + */ + @Default + @Parameter + public double compression() { + return 100.0; + } + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } + + @Check + final void checkPercentile() { + if (percentile() < 0.0 || percentile() > 1.0) { + throw new IllegalArgumentException("Percentile must be in range [0.0, 1.0]"); + } + } + + @Check + final void checkCompression() { + if (compression() < 1.0) { + throw new IllegalArgumentException("Compression must be greater than or equal to 1.0"); + } + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java index 6224b2afa57..b23d01c91ce 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java @@ -25,6 +25,11 @@ public void visit(AggSpecAbsSum absSum) { out = Collections.emptySet(); } + @Override + public void visit(AggSpecApproximatePercentile pct) { + out = Collections.emptySet(); + } + @Override public void visit(AggSpecCountDistinct countDistinct) { out = Collections.emptySet(); @@ -100,6 +105,11 @@ public void visit(AggSpecSum sum) { out = Collections.emptySet(); } + @Override + public void visit(AggSpecTDigest tDigest) { + out = Collections.emptySet(); + } + @Override public void visit(AggSpecUnique unique) { out = Collections.emptySet(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java new file mode 100644 index 00000000000..9e5fc77c1d1 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java @@ -0,0 +1,56 @@ +package io.deephaven.api.agg.spec; + +import io.deephaven.annotations.BuildableStyle; +import org.immutables.value.Value.Check; +import org.immutables.value.Value.Default; +import org.immutables.value.Value.Immutable; + +/** + *

+ * Aggregates input column into a TDigest output column with the specified compression. + *

+ * May be used to implement parallel percentile calculations by splitting inputs and accumulating results into a single + * downstream TDigest. + *

+ * May only be used on static or add-only tables. + */ +@Immutable +@BuildableStyle +public abstract class AggSpecTDigest implements AggSpec { + + public static AggSpecTDigest of() { + return ImmutableAggSpecTDigest.builder().build(); + } + + public static AggSpecTDigest of(double compression) { + return ImmutableAggSpecTDigest.builder().compression(compression).build(); + } + + @Override + public final String description() { + return String.format("TDigest with compression %.2f", compression()); + } + + /** + * T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large. + * + * @return The T-Digest compression factor + */ + @Default + public double compression() { + return 100.0; + } + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } + + @Check + final void checkCompression() { + if (compression() < 1.0) { + throw new IllegalArgumentException("Compression must be greater than or equal to 1.0"); + } + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/util/PercentileOutput.java b/table-api/src/main/java/io/deephaven/api/agg/util/PercentileOutput.java new file mode 100644 index 00000000000..64444e45f06 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/util/PercentileOutput.java @@ -0,0 +1,47 @@ +package io.deephaven.api.agg.util; + +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.ColumnName; +import org.immutables.value.Value; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +/** + * Percentile and output column pair, used when specifying that a given input column should be aggregated into multiple + * exact or approximate percentiles. + */ +@Immutable +@SimpleStyle +public abstract class PercentileOutput { + + public static PercentileOutput of(double percentile, ColumnName output) { + return ImmutablePercentileOutput.of(percentile, output); + } + + public static PercentileOutput of(double percentile, String output) { + return of(percentile, ColumnName.of(output)); + } + + /** + * Percentile. Must be in range [0.0, 1.0]. + * + * @return The percentile + */ + @Parameter + public abstract double percentile(); + + /** + * Output {@link ColumnName column name}. + * + * @return The output {@link ColumnName column name} + */ + @Parameter + public abstract ColumnName output(); + + @Value.Check + final void checkPercentile() { + if (percentile() < 0.0 || percentile() > 1.0) { + throw new IllegalArgumentException("Percentile must be in range [0.0, 1.0]"); + } + } +} From b6145182d6135929a6d3c746bc405a66f83fadaa Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 28 Jan 2022 23:30:13 -0500 Subject: [PATCH 22/44] Some build fixes and cleanups. --- .../table/impl/AggAllByCopyAttributes.java | 8 ++ .../engine/table/impl/AggAllByUseTable.java | 12 ++ .../table/impl/by/AggregationProcessor.java | 126 +++++++++++++++--- .../client/impl/BatchTableRequestBuilder.java | 2 +- .../qst/table/AggAllByExclusions.java | 12 ++ .../io/deephaven/api/agg/Aggregations.java | 3 +- .../io/deephaven/api/agg/spec/AggSpec.java | 2 +- .../spec/AggSpecApproximatePercentile.java | 2 +- .../api/agg/spec/AggSpecColumnReferences.java | 2 +- .../api/agg/spec/AggSpecTDigest.java | 2 +- 10 files changed, 147 insertions(+), 24 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java index 6a066a8a382..21cdf079e1b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java @@ -2,6 +2,7 @@ import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecAbsSum; +import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; import io.deephaven.api.agg.spec.AggSpecAvg; import io.deephaven.api.agg.spec.AggSpecCountDistinct; import io.deephaven.api.agg.spec.AggSpecDistinct; @@ -17,6 +18,7 @@ import io.deephaven.api.agg.spec.AggSpecSortedLast; import io.deephaven.api.agg.spec.AggSpecStd; import io.deephaven.api.agg.spec.AggSpecSum; +import io.deephaven.api.agg.spec.AggSpecTDigest; import io.deephaven.api.agg.spec.AggSpecUnique; import io.deephaven.api.agg.spec.AggSpecVar; import io.deephaven.api.agg.spec.AggSpecWAvg; @@ -39,6 +41,9 @@ public AggAllByCopyAttributes(BaseTable parent, Table result) { @Override public void visit(AggSpecAbsSum absSum) {} + @Override + public void visit(AggSpecApproximatePercentile approxPct) {} + @Override public void visit(AggSpecCountDistinct countDistinct) {} @@ -88,6 +93,9 @@ public void visit(AggSpecStd std) {} @Override public void visit(AggSpecSum sum) {} + @Override + public void visit(AggSpecTDigest tDigest) {} + @Override public void visit(AggSpecUnique unique) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java index 21f3ecbd2a0..5be88f67e43 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java @@ -2,6 +2,7 @@ import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecAbsSum; +import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; import io.deephaven.api.agg.spec.AggSpecAvg; import io.deephaven.api.agg.spec.AggSpecCountDistinct; import io.deephaven.api.agg.spec.AggSpecDistinct; @@ -17,6 +18,7 @@ import io.deephaven.api.agg.spec.AggSpecSortedLast; import io.deephaven.api.agg.spec.AggSpecStd; import io.deephaven.api.agg.spec.AggSpecSum; +import io.deephaven.api.agg.spec.AggSpecTDigest; import io.deephaven.api.agg.spec.AggSpecUnique; import io.deephaven.api.agg.spec.AggSpecVar; import io.deephaven.api.agg.spec.AggSpecWAvg; @@ -55,6 +57,11 @@ public void visit(AggSpecAbsSum absSum) { drop(); } + @Override + public void visit(AggSpecApproximatePercentile approxPct) { + drop(); + } + @Override public void visit(AggSpecCountDistinct countDistinct) { drop(); @@ -130,6 +137,11 @@ public void visit(AggSpecSum sum) { drop(); } + @Override + public void visit(AggSpecTDigest tDigest) { + drop(); + } + @Override public void visit(AggSpecUnique unique) { keep(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index a4fc2a4df83..6f03d9c5333 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -2,19 +2,101 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; -import io.deephaven.api.agg.*; -import io.deephaven.api.agg.spec.*; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.AggregationPairs; +import io.deephaven.api.agg.ColumnAggregation; +import io.deephaven.api.agg.ColumnAggregations; +import io.deephaven.api.agg.Count; +import io.deephaven.api.agg.FirstRowKey; +import io.deephaven.api.agg.LastRowKey; +import io.deephaven.api.agg.Pair; +import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.api.agg.spec.AggSpecAbsSum; +import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; +import io.deephaven.api.agg.spec.AggSpecAvg; +import io.deephaven.api.agg.spec.AggSpecCountDistinct; +import io.deephaven.api.agg.spec.AggSpecDistinct; +import io.deephaven.api.agg.spec.AggSpecFirst; +import io.deephaven.api.agg.spec.AggSpecFormula; +import io.deephaven.api.agg.spec.AggSpecGroup; +import io.deephaven.api.agg.spec.AggSpecLast; +import io.deephaven.api.agg.spec.AggSpecMax; +import io.deephaven.api.agg.spec.AggSpecMedian; +import io.deephaven.api.agg.spec.AggSpecMin; +import io.deephaven.api.agg.spec.AggSpecPercentile; +import io.deephaven.api.agg.spec.AggSpecSortedFirst; +import io.deephaven.api.agg.spec.AggSpecSortedLast; +import io.deephaven.api.agg.spec.AggSpecStd; +import io.deephaven.api.agg.spec.AggSpecSum; +import io.deephaven.api.agg.spec.AggSpecTDigest; +import io.deephaven.api.agg.spec.AggSpecUnique; +import io.deephaven.api.agg.spec.AggSpecVar; +import io.deephaven.api.agg.spec.AggSpecWAvg; +import io.deephaven.api.agg.spec.AggSpecWSum; import io.deephaven.chunk.ChunkType; import io.deephaven.chunk.attributes.Values; import io.deephaven.datastructures.util.SmartKey; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.*; +import io.deephaven.engine.table.ChunkSource; +import io.deephaven.engine.table.ColumnDefinition; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.MatchPair; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.EmptyTableMap; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.ReverseLookup; +import io.deephaven.engine.table.impl.RollupInfo; +import io.deephaven.engine.table.impl.TupleSourceFactory; import io.deephaven.engine.table.impl.by.rollup.NullColumns; import io.deephaven.engine.table.impl.by.rollup.Partition; import io.deephaven.engine.table.impl.by.rollup.RollupAggregation; -import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.*; -import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.*; -import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.*; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ByteChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ByteRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.CharChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.CharRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.DoubleChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.DoubleRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.FloatChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.FloatRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.IntChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.IntRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.LongChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.LongRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ObjectChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ObjectRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ShortChunkedCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ShortRollupCountDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.ByteChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.ByteRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.CharChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.CharRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.DoubleChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.DoubleRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.FloatChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.FloatRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.IntChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.IntRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.LongChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.LongRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.ObjectChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.ObjectRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.ShortChunkedDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct.ShortRollupDistinctOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ByteChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ByteRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.CharChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.CharRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.DoubleChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.DoubleRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.FloatChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.FloatRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.IntChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.IntRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.LongChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.LongRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ObjectChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ObjectRollupUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ShortChunkedUniqueOperator; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ShortRollupUniqueOperator; import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; import io.deephaven.engine.table.impl.by.ssmpercentile.SsmChunkedPercentileOperator; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; @@ -29,14 +111,21 @@ import java.math.BigDecimal; import java.math.BigInteger; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.ToIntFunction; import java.util.stream.Collectors; import java.util.stream.Stream; -import static io.deephaven.datastructures.util.CollectionUtil.*; +import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_DOUBLE_ARRAY; +import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; +import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY; import static io.deephaven.engine.table.ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY; import static io.deephaven.engine.table.Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE; import static io.deephaven.engine.table.Table.REVERSE_LOOKUP_ATTRIBUTE; @@ -45,8 +134,15 @@ import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; import static io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY; import static io.deephaven.engine.table.impl.by.RollupConstants.*; -import static io.deephaven.util.QueryConstants.*; -import static io.deephaven.util.type.TypeUtils.*; +import static io.deephaven.util.QueryConstants.NULL_BYTE; +import static io.deephaven.util.QueryConstants.NULL_DOUBLE; +import static io.deephaven.util.QueryConstants.NULL_FLOAT; +import static io.deephaven.util.QueryConstants.NULL_INT; +import static io.deephaven.util.QueryConstants.NULL_LONG; +import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.util.type.TypeUtils.getBoxedType; +import static io.deephaven.util.type.TypeUtils.isNumeric; +import static io.deephaven.util.type.TypeUtils.unbox; /** * Conversion tool to generate an {@link AggregationContextFactory} for a collection of {@link Aggregation @@ -518,8 +614,8 @@ public void visit(@NotNull final AggSpecAbsSum absSum) { } @Override - public void visit(@NotNull final AggSpecApproximatePercentile pct) { - addApproximatePercentileOperators(pct.percentile(), pct.compression()); + public void visit(@NotNull final AggSpecApproximatePercentile approxPct) { + addApproximatePercentileOperators(approxPct.percentile(), approxPct.compression()); } @Override @@ -1471,10 +1567,6 @@ private static IterativeChunkedAggregationOperator makeAvgOperator( return new BigIntegerChunkedAvgOperator(name, exposeInternal); } else if (type == BigDecimal.class) { return new BigDecimalChunkedAvgOperator(name, exposeInternal); - } else if (AvgState.class.isAssignableFrom(type)) { - throw new UnsupportedOperationException(); - } else if (AvgStateWithNan.class.isAssignableFrom(type)) { - throw new UnsupportedOperationException(); } throw new UnsupportedOperationException("Unsupported type " + type); } diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index 7c179857dbf..16a29dbd708 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -680,7 +680,7 @@ public void visit(AggSpecAbsSum absSum) { } @Override - public void visit(AggSpecApproximatePercentile pct) { + public void visit(AggSpecApproximatePercentile approxPct) { throw new UnsupportedOperationException( "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } diff --git a/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java b/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java index ad8d35ed17e..57e8660ba19 100644 --- a/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java +++ b/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java @@ -5,6 +5,7 @@ import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpec.Visitor; import io.deephaven.api.agg.spec.AggSpecAbsSum; +import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; import io.deephaven.api.agg.spec.AggSpecAvg; import io.deephaven.api.agg.spec.AggSpecCountDistinct; import io.deephaven.api.agg.spec.AggSpecDistinct; @@ -20,6 +21,7 @@ import io.deephaven.api.agg.spec.AggSpecSortedLast; import io.deephaven.api.agg.spec.AggSpecStd; import io.deephaven.api.agg.spec.AggSpecSum; +import io.deephaven.api.agg.spec.AggSpecTDigest; import io.deephaven.api.agg.spec.AggSpecUnique; import io.deephaven.api.agg.spec.AggSpecVar; import io.deephaven.api.agg.spec.AggSpecWAvg; @@ -55,6 +57,11 @@ public void visit(AggSpecAbsSum absSum) { out = Collections.emptySet(); } + @Override + public void visit(AggSpecApproximatePercentile approxPct) { + out = Collections.emptySet(); + } + @Override public void visit(AggSpecCountDistinct countDistinct) { out = Collections.emptySet(); @@ -130,6 +137,11 @@ public void visit(AggSpecSum sum) { out = Collections.emptySet(); } + @Override + public void visit(AggSpecTDigest tDigest) { + out = Collections.emptySet(); + } + @Override public void visit(AggSpecUnique unique) { out = Collections.emptySet(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java index b829d182612..989bf6f8eb6 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregations.java @@ -18,8 +18,7 @@ public static Aggregations.Builder builder() { return ImmutableAggregations.builder(); } - @Parameter - public abstract List aggregations(); + public abstract List aggregations(); @Override public final V walk(V visitor) { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index fb6d62cbb95..150cb01d57a 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -163,7 +163,7 @@ static AggSpecWSum wsum(String weightColumn) { interface Visitor { void visit(AggSpecAbsSum absSum); - void visit(AggSpecApproximatePercentile pct); + void visit(AggSpecApproximatePercentile approxPct); void visit(AggSpecCountDistinct countDistinct); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java index 11873411172..b66c6dde6bf 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java @@ -12,7 +12,7 @@ */ @Immutable @BuildableStyle -public abstract class AggSpecApproximatePercentile implements AggSpec { +public abstract class AggSpecApproximatePercentile extends AggSpecBase { public static AggSpecApproximatePercentile of(double percentile) { return ImmutableAggSpecApproximatePercentile.builder().percentile(percentile).build(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java index b23d01c91ce..068fca2f0f2 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java @@ -26,7 +26,7 @@ public void visit(AggSpecAbsSum absSum) { } @Override - public void visit(AggSpecApproximatePercentile pct) { + public void visit(AggSpecApproximatePercentile approxPct) { out = Collections.emptySet(); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java index 9e5fc77c1d1..0d34bd7c3d8 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java @@ -16,7 +16,7 @@ */ @Immutable @BuildableStyle -public abstract class AggSpecTDigest implements AggSpec { +public abstract class AggSpecTDigest extends AggSpecBase { public static AggSpecTDigest of() { return ImmutableAggSpecTDigest.builder().build(); From 929e2198ef72fca217def130e16f5a2e698ad4ee Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Fri, 28 Jan 2022 23:50:55 -0500 Subject: [PATCH 23/44] Commit previously missing file. --- .../engine/table/impl/by/RollupConstants.java | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java new file mode 100644 index 00000000000..c324c631dda --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java @@ -0,0 +1,62 @@ +package io.deephaven.engine.table.impl.by; + +/** + * Implementation constants for {@link io.deephaven.engine.table.Table#rollup} support. + */ +public final class RollupConstants { + + private RollupConstants() { + } + + /** + * Marker suffiix for rollup-internal column names. + */ + public static final String ROLLUP_COLUMN_SUFFIX = "__ROLLUP__"; + + /** + * Prefix for row redirection columns. + */ + static final String ROW_REDIRECTION_PREFIX = "RowRedirection_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for SSM + * columns used in "distinct", "count distinct", and "unique" rollup aggregations. + */ + public static final String ROLLUP_DISTINCT_SSM_COLUMN_ID = "_SSM_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for running + * sum columns used in rollup aggregations. + */ + static final String ROLLUP_RUNNING_SUM_COLUMN_ID = "_RS_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for running + * sum of squares columns used in rollup aggregations. + */ + static final String ROLLUP_RUNNING_SUM2_COLUMN_ID = "_RS2_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for non-null + * count columns used in rollup aggregations. + */ + static final String ROLLUP_NONNULL_COUNT_COLUMN_ID = "_NNC_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for NaN count + * columns used in rollup aggregations. + */ + static final String ROLLUP_NAN_COUNT_COLUMN_ID = "_NaNC_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for positive + * infinity count columns used in rollup aggregations. + */ + static final String ROLLUP_PI_COUNT_COLUMN_ID = "_PIC_"; + + /** + * Middle column name component (between source column name and {@link #ROLLUP_COLUMN_SUFFIX suffix}) for negative + * infinity count columns used in rollup aggregations. + */ + static final String ROLLUP_NI_COUNT_COLUMN_ID = "_NIC_"; +} From 4b1060bcca48faa3ab5b718d8e99ea1c85fbe805 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 00:05:27 -0500 Subject: [PATCH 24/44] Three more fixes for compilation. --- .../deephaven/treetable/RollupSnapshotImpl.java | 3 +-- .../java/io/deephaven/plot/util/PlotUtils.java | 7 +++---- .../table/impl/by/AggregationProcessor.java | 12 ++++-------- .../table/impl/by/rollup/NullColumns.java | 5 ++--- .../table/impl/by/rollup/RollupAggregation.java | 2 +- .../table/impl/TestConcurrentInstantiation.java | 17 +++++++++-------- 6 files changed, 20 insertions(+), 26 deletions(-) diff --git a/ClientSupport/src/main/java/io/deephaven/treetable/RollupSnapshotImpl.java b/ClientSupport/src/main/java/io/deephaven/treetable/RollupSnapshotImpl.java index d194008a000..469fd3bd426 100644 --- a/ClientSupport/src/main/java/io/deephaven/treetable/RollupSnapshotImpl.java +++ b/ClientSupport/src/main/java/io/deephaven/treetable/RollupSnapshotImpl.java @@ -2,7 +2,6 @@ import io.deephaven.base.verify.Require; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.QueryTable; import io.deephaven.engine.updategraph.NotificationQueue; import io.deephaven.engine.table.MatchPair; import io.deephaven.engine.util.string.StringUtils; @@ -162,7 +161,7 @@ private HierarchicalTable applyFilters(@NotNull HierarchicalTable table) { final Table source = Require.neqNull(table.getSourceTable(), "Hierarchical source table"); final RollupInfo info = getInfo(); - return (HierarchicalTable) ((QueryTable) source.where(filters)).rollup(info.factory, + return (HierarchicalTable) source.where(filters).rollup(info.aggregations, info.getLeafType() == RollupInfo.LeafType.Constituent, info.getSelectColumns()); } diff --git a/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java b/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java index 1685303c928..98e2378cde1 100644 --- a/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java +++ b/Plot/src/main/java/io/deephaven/plot/util/PlotUtils.java @@ -8,8 +8,6 @@ import io.deephaven.api.agg.Aggregation; import io.deephaven.base.verify.Require; import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.select.SelectColumn; import io.deephaven.plot.ChartImpl; import io.deephaven.plot.datasets.category.CategoryDataSeries; import io.deephaven.plot.datasets.data.*; @@ -40,6 +38,7 @@ import java.util.stream.IntStream; import static io.deephaven.api.agg.Aggregation.AggCount; +import static io.deephaven.api.agg.Aggregation.AggLast; import static io.deephaven.util.QueryConstants.*; import static io.deephaven.function.IntegerNumericPrimitives.abs; @@ -722,9 +721,9 @@ public static Table createCategoryTable(final Table t, final String[] catColumns // We need to do the equivalent of LastBy wrt. to columns included, or we have a chance to break ACLs final List lastColumns = t.getDefinition().getColumnNames(); lastColumns.removeAll(Arrays.asList(catColumns)); - final Table result = ((QueryTable) t).by( + final Table result = t.aggBy( createCategoryAggs(AggLast(lastColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY))), - SelectColumn.from(Selectable.from(catColumns))); + Selectable.from(catColumns)); // We must explicitly copy attributes because we are doing a modified manual first/lastBy which will not // automatically do the copy. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 6f03d9c5333..2cac102bce5 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -203,14 +203,10 @@ public static AggregationContextFactory forRollupBase( public static AggregationContextFactory forRollupReaggregated( @NotNull final Collection aggregations, @NotNull final Map> nullColumns) { - // @formatter:off - final Collection reaggregations = - Stream.of( - Stream.of(NullColumns.from(nullColumns)), - aggregations.stream(), - Stream.of(Partition.of(false)) - ).flatMap(Function.identity()).collect(Collectors.toList()); - // @formatter:on + final Collection reaggregations = new ArrayList<>(aggregations.size() + 2); + reaggregations.add(NullColumns.from(nullColumns)); + reaggregations.addAll(aggregations); + reaggregations.add(Partition.of(false)); return new AggregationProcessor(reaggregations, Type.ROLLUP_REAGGREGATED); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java index dc6122ce984..a29b5731e7e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/NullColumns.java @@ -17,7 +17,7 @@ public static NullColumns of(String name, Class type) { } public static NullColumns from(Map> resultColumns) { - return builder().putResultColumns(resultColumns).build(); + return builder().putAllResultColumns(resultColumns).build(); } public static Builder builder() { @@ -41,10 +41,9 @@ final void checkNonEmpty() { } public interface Builder { - Builder putResultColumns(String name, Class type); - Builder putResultColumns(Map> resultColumns); + Builder putAllResultColumns(Map> resultColumns); NullColumns build(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java index ee1c8efadf4..43f276fc16f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.java @@ -10,7 +10,7 @@ public interface RollupAggregation extends Aggregation { static RollupAggregation nullColumns(Map> resultColumns) { - return NullColumns.builder().putResultColumns(resultColumns).build(); + return NullColumns.from(resultColumns); } static RollupAggregation partition(boolean includeConstituents) { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java index ee8d496c185..9cffe5b817b 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestConcurrentInstantiation.java @@ -7,7 +7,6 @@ import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.select.SourceColumn; import io.deephaven.engine.table.lang.QueryLibrary; import io.deephaven.engine.updategraph.UpdateGraphProcessor; import io.deephaven.engine.table.impl.select.MatchPairFactory; @@ -16,7 +15,6 @@ import io.deephaven.engine.util.TableDiff; import io.deephaven.engine.util.TableTools; import io.deephaven.engine.liveness.LivenessScopeStack; -import io.deephaven.engine.table.impl.by.*; import io.deephaven.engine.table.impl.remote.ConstructSnapshot; import io.deephaven.engine.table.impl.select.ConditionFilter; import io.deephaven.engine.table.impl.select.DisjunctiveFilter; @@ -43,6 +41,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.function.UnaryOperator; + +import org.jetbrains.annotations.NotNull; import org.junit.experimental.categories.Category; import static io.deephaven.api.agg.Aggregation.*; @@ -1134,20 +1134,21 @@ public void testCountBy() throws Exception { testByConcurrent(t -> t.varBy("KeyColumn")); } + private static T setAddOnly(@NotNull final T table) { + table.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, true); + return table; + } + public void testMinMaxBy() throws Exception { testByConcurrent(t -> t.maxBy("KeyColumn")); testByConcurrent(t -> t.minBy("KeyColumn")); - testByConcurrent(t -> ((QueryTable) t).by(new AddOnlyMinMaxBySpecImpl(true), new SourceColumn("KeyColumn")), - true, false, false, true); - testByConcurrent(t -> ((QueryTable) t).by(new AddOnlyMinMaxBySpecImpl(false), new SourceColumn("KeyColumn")), - true, false, false, true); + testByConcurrent(t -> setAddOnly(t).minBy("KeyColumn"), true, false, false, true); + testByConcurrent(t -> setAddOnly(t).maxBy("KeyColumn"), true, false, false, true); } public void testFirstLastBy() throws Exception { testByConcurrent(t -> t.firstBy("KeyColumn")); testByConcurrent(t -> t.lastBy("KeyColumn")); - testByConcurrent(t -> ((QueryTable) t).by(new TrackingFirstBySpecImpl(), new SourceColumn("KeyColumn"))); - testByConcurrent(t -> ((QueryTable) t).by(new TrackingLastBySpecImpl(), new SourceColumn("KeyColumn"))); } public void testSortedFirstLastBy() throws Exception { From ddaa05a870a48b49cf852e82b2b04136fed5aedf Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 00:05:50 -0500 Subject: [PATCH 25/44] Delete many legacy files. --- .../engine/table/impl/by/AbsSumSpec.java | 46 - .../impl/by/AddOnlyMinMaxBySpecImpl.java | 15 - .../engine/table/impl/by/AggType.java | 42 - .../impl/by/AggregationElementAdapter.java | 57 - .../table/impl/by/AggregationFactory.java | 1064 ----------------- .../table/impl/by/AggregationFormulaSpec.java | 57 - .../table/impl/by/AggregationGroupSpec.java | 14 - .../table/impl/by/AggregationMemoKey.java | 7 - .../engine/table/impl/by/AggregationSpec.java | 30 - .../table/impl/by/AggregationSpecAdapter.java | 146 --- .../engine/table/impl/by/AvgSpec.java | 46 - .../engine/table/impl/by/AvgState.java | 60 - .../engine/table/impl/by/AvgStateWithNan.java | 41 - .../engine/table/impl/by/BaseConverter.java | 19 - .../impl/by/CountAggregationFactory.java | 36 - .../engine/table/impl/by/CountBySpecImpl.java | 65 - .../table/impl/by/CountDistinctSpec.java | 56 - .../engine/table/impl/by/DistinctSpec.java | 57 - .../engine/table/impl/by/FirstBySpecImpl.java | 52 - .../table/impl/by/IterativeIndexSpec.java | 90 -- .../table/impl/by/IterativeOperatorSpec.java | 191 --- .../impl/by/KeyOnlyFirstOrLastBySpec.java | 76 -- .../engine/table/impl/by/LastBySpecImpl.java | 50 - .../table/impl/by/MinMaxBySpecImpl.java | 80 -- .../by/MinMaxIterativeOperatorFactory.java | 31 - .../by/NonKeyColumnAggregationFactory.java | 75 -- .../impl/by/NullAggregationSpecImpl.java | 10 - .../table/impl/by/PercentileBySpecImpl.java | 64 - .../PercentileIterativeOperatorFactory.java | 29 - .../table/impl/by/PreviousStateProvider.java | 5 - .../impl/by/ReaggregatableStatefactory.java | 27 - .../by/ReaggregationIterativeOperator.java | 11 - .../engine/table/impl/by/RowStatus.java | 9 - .../table/impl/by/SelectDistinctSpecImpl.java | 19 - .../engine/table/impl/by/SortedFirstBy.java | 11 - ...SortedFirstOrLastByAggregationFactory.java | 115 -- .../by/SortedFirstOrLastByFactoryImpl.java | 80 -- .../engine/table/impl/by/SortedLastBy.java | 11 - .../engine/table/impl/by/StdSpec.java | 46 - .../engine/table/impl/by/StdState.java | 63 - .../engine/table/impl/by/StdStateWithNan.java | 37 - .../engine/table/impl/by/SumSpec.java | 45 - .../impl/by/TrackingFirstBySpecImpl.java | 50 - .../table/impl/by/TrackingLastBySpecImpl.java | 50 - .../engine/table/impl/by/UniqueSpec.java | 100 -- .../engine/table/impl/by/VarSpec.java | 45 - .../engine/table/impl/by/VarState.java | 63 - .../engine/table/impl/by/VarStateWithNan.java | 37 - .../impl/by/WeightedAverageSpecImpl.java | 52 - .../table/impl/by/WeightedSumSpecImpl.java | 69 -- 50 files changed, 3551 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AbsSumSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationGroupSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationMemoKey.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgState.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgStateWithNan.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountDistinctSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/DistinctSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/NullAggregationSpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/PreviousStateProvider.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/RowStatus.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstBy.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedLastBy.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdState.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdStateWithNan.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/SumSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarState.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarStateWithNan.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AbsSumSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AbsSumSpec.java deleted file mode 100644 index 367ea7e4857..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AbsSumSpec.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - - -/** - * Factory for iterative absolute sum aggregations. - */ -public class AbsSumSpec extends IterativeOperatorSpec { - public AbsSumSpec() {} - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getAbsSumChunked(type, name); - } - - private static final AggregationMemoKey ABS_SUM_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return ABS_SUM_INSTANCE; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return this; - } - - @Override - public String toString() { - return "AbsSum"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.java deleted file mode 100644 index 0f63966349c..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.java +++ /dev/null @@ -1,15 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * If you've got a table that is grow only, this will do a min/max calculation without requiring any state. The - * limitation is that if you modify or remove a row it will throw an UnsupportedOperationException. - */ -public class AddOnlyMinMaxBySpecImpl extends MinMaxBySpecImpl { - public AddOnlyMinMaxBySpecImpl(boolean minimum) { - super(minimum, true); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java deleted file mode 100644 index 94832087ef7..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggType.java +++ /dev/null @@ -1,42 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TotalsTableBuilder; - -/** - * Enumeration representing valid aggregation types for {@link TotalsTableBuilder}. - */ -public enum AggType { - /** Return the number of rows in each group. */ - Count, - /** Return the minimum value of each group. */ - Min, - /** Return the maximum value of each group. */ - Max, - /** Return the sum of values in each group. */ - Sum, - /** Return the sum of absolute values in each group. */ - AbsSum, - /** Return the variance of values in each group. */ - Var, - /** Return the average of values in each group. */ - Avg, - /** Return the standard deviation of each group. */ - Std, - /** Return the first value of each group. */ - First, - /** Return the last value of each group. */ - Last, - /** Return the values of each group as a Vector. */ - Group, - /** Return the number of unique values in each group */ - CountDistinct, - /** Collect the distinct items from the column */ - Distinct, - /** - * Display the singular value from the column if it is unique, or a default value if none are present, or it is not - * unique - */ - Unique, - /** Only valid in a TotalsTableBuilder to indicate we should not perform any aggregation. */ - Skip -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java deleted file mode 100644 index 9706f9a330b..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationElementAdapter.java +++ /dev/null @@ -1,57 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.api.agg.*; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement; -import io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElementImpl; - -import java.util.Objects; - -/** - * Utility for converting an {@link Aggregation} to an {@link AggregationElement}. - */ -class AggregationElementAdapter implements Aggregation.Visitor { - - public static AggregationElement of(Aggregation aggregation) { - return aggregation.walk(new AggregationElementAdapter()).out(); - } - - private AggregationElement out; - - public AggregationElement out() { - return Objects.requireNonNull(out); - } - - @Override - public void visit(Count count) { - out = new AggregationFactory.CountAggregationElement(count.column().name()); - } - - @Override - public void visit(FirstRowKey firstRowKey) { - out = new AggregationElementImpl( - new KeyOnlyFirstOrLastBySpec(firstRowKey.column().name(), AggType.First), - MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY); - } - - @Override - public void visit(LastRowKey lastRowKey) { - out = new AggregationElementImpl( - new KeyOnlyFirstOrLastBySpec(lastRowKey.column().name(), AggType.Last), - MatchPair.ZERO_LENGTH_MATCH_PAIR_ARRAY); - } - - @Override - public void visit(ColumnAggregation columnAgg) { - final AggregationSpec spec = AggregationSpecAdapter.of(columnAgg.spec()); - final MatchPair pair = MatchPair.of(columnAgg.pair()); - out = new AggregationElementImpl(spec, pair); - } - - @Override - public void visit(ColumnAggregations columnAggs) { - final AggregationSpec spec = AggregationSpecAdapter.of(columnAggs.spec()); - final MatchPair[] pairs = MatchPair.fromPairs(columnAggs.pairs()); - out = new AggregationElementImpl(spec, pairs); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java deleted file mode 100644 index fd6e9132d12..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFactory.java +++ /dev/null @@ -1,1064 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.AggregationOptimizer; -import io.deephaven.base.verify.Assert; -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.datastructures.util.SmartKey; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.ChunkType; -import io.deephaven.engine.table.*; -import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; -import io.deephaven.engine.table.impl.select.MatchPairFactory; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.engine.table.impl.sources.SingleValueObjectColumnSource; -import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; -import io.deephaven.time.DateTime; -import org.jetbrains.annotations.NotNull; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.util.*; -import java.util.function.ToIntFunction; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -/** - * The AggregationProcessor combines one or more {@link Aggregation aggregations} into an - * {@link AggregationContextFactory} for use internally by the implementation of {@link Table#aggBy}. - */ -public class AggregationFactory implements AggregationSpec { - - private final List underlyingAggregations = new ArrayList<>(); - private final boolean isRollup; - private final boolean secondLevel; - - /** - * Create a factory for performing rollups. - */ - public AggregationFactory rollupFactory() { - // we want to leave off the null value column source for children; but add a by external combo for the rollup - return new AggregationFactory( - Stream.concat(underlyingAggregations.subList(0, underlyingAggregations.size() - 1).stream().map(x -> { - final AggregationSpec underlyingStateFactory = x.getSpec(); - Assert.assertion(underlyingStateFactory instanceof ReaggregatableStatefactory, - "underlyingStateFactory instanceof ReaggregatableStatefactory", underlyingStateFactory, - "UnderlyingStateFactory"); - - // noinspection ConstantConditions - final ReaggregatableStatefactory reaggregatableStatefactory = - (ReaggregatableStatefactory) underlyingStateFactory; - - Assert.assertion(reaggregatableStatefactory.supportsRollup(), - "((ReaggregatableStatefactory)x.getUnderlyingStateFactory()).supportsRollup()", - underlyingStateFactory, "UnderlyingStateFactory"); - final ReaggregatableStatefactory factory = reaggregatableStatefactory.rollupFactory(); - - final List leftColumns = new ArrayList<>(); - Collections.addAll(leftColumns, MatchPair.getLeftColumns(x.getResultPairs())); - - return Agg(factory, leftColumns.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY)); - }), Stream.of(new PartitionAggregationElement(false))).collect(Collectors.toList()), true, true); - } - - public AggregationFactory forRollup(boolean includeConstituents) { - final List newUnderliers = - underlyingAggregations.stream().map(AggregationElement::forRollup).collect(Collectors.toList()); - newUnderliers.add(includeConstituents ? new PartitionAggregationElement(true) - : new NullAggregationElement(Collections.singletonMap(RollupInfo.ROLLUP_COLUMN, Object.class))); - return new AggregationFactory(newUnderliers, true, false); - } - - /** - * Create a new factory that will have columns with all null values. - * - * Used by rollup to empty out unused grouping columns. - * - * @param nullColumns a map of column names to types. - * - * @return a new AggregationFactory that will produce null values for the given columns. - */ - public AggregationFactory withNulls(Map> nullColumns) { - final List newAggregations = new ArrayList<>(underlyingAggregations.size() + 1); - newAggregations.add(new NullAggregationElement(nullColumns)); - newAggregations.addAll(underlyingAggregations); - return new AggregationFactory(newAggregations, isRollup, secondLevel); - } - - private static final String[] ROLLUP_KEY_COLUMNS = {RollupInfo.ROLLUP_COLUMN}; - - private String[] getKeyColumns() { - return ROLLUP_KEY_COLUMNS; - } - - public interface AggregationElement { - - /** - * Converts an {@link Aggregation} to an {@link AggregationElement}. - * - * @param aggregation The {@link Aggregation aggregation} - * @return The {@link AggregationElement aggregation element} - */ - static AggregationElement of(Aggregation aggregation) { - return AggregationElementAdapter.of(aggregation); - } - - /** - * Converts and optimizes the aggregations, collapsing relevant aggregations into single - * {@link AggregationElement elements} where applicable. - * - *

- * Note: due to the optimization, the aggregation elements may not be in the same order as specified in - * {@code aggregations}. - * - * @param aggregations The {@link Aggregation aggregation} - * @return A list of {@link AggregationElement aggregation elements} - * @see AggregationOptimizer#of(Collection) - * @see #of(Aggregation) - * @see #convert(Collection) - */ - static List optimizeAndConvert(Collection aggregations) { - return convert(AggregationOptimizer.of(aggregations)); - } - - /** - * Converts the aggregations leaving singular aggregations as they are. - * - *

- * Note: The results will preserve the intended order of the inputs. - * - * @param aggregations The {@link Aggregation aggregation} - * @return A list of {@link AggregationElement aggregation elements} - * @see #of(Aggregation) - * @see #optimizeAndConvert(Collection) - */ - static List convert(Collection aggregations) { - final List out = new ArrayList<>(aggregations.size()); - for (Aggregation aggregation : aggregations) { - out.add(of(aggregation)); - } - return out; - } - - AggregationSpec getSpec(); - - String[] getSourceColumns(); - - MatchPair[] getResultPairs(); - - AggregationElement forRollup(); - - AggregationMemoKey getMemoKey(); - } - - static public class AggregationElementImpl implements AggregationElement { - private final MatchPair[] matchPairs; - private final String[] rightColumns; - private final AggregationSpec spec; - - public AggregationElementImpl(final AggregationSpec spec, final String... matchPairs) { - this(spec, MatchPairFactory.getExpressions(matchPairs)); - } - - @SuppressWarnings("unused") - public AggregationElementImpl(final AggregationSpec spec, final Collection matchPairs) { - this(spec, MatchPairFactory.getExpressions(matchPairs)); - } - - AggregationElementImpl(final AggregationSpec spec, final MatchPair... matchPairs) { - this.matchPairs = matchPairs; - this.spec = spec; - this.rightColumns = new String[matchPairs.length]; - for (int ii = 0; ii < matchPairs.length; ++ii) { - this.rightColumns[ii] = this.matchPairs[ii].rightColumn; - } - } - - @Override - public AggregationSpec getSpec() { - return spec; - } - - @Override - public String[] getSourceColumns() { - return rightColumns; - } - - @Override - public MatchPair[] getResultPairs() { - return matchPairs; - } - - @Override - public AggregationElement forRollup() { - if (!(spec instanceof ReaggregatableStatefactory)) { - throw new UnsupportedOperationException( - "Not a reaggregatable state factory: " + spec); - } - if (!((ReaggregatableStatefactory) spec).supportsRollup()) { - throw new UnsupportedOperationException( - "Underlying state factory does not support rollup: " + spec); - } - return new AggregationElementImpl(((ReaggregatableStatefactory) spec).forRollup(), matchPairs); - } - - @Override - public AggregationMemoKey getMemoKey() { - return getSpec().getMemoKey(); - } - - @Override - public String toString() { - return "Agg{" + spec + ", " + Arrays.toString(matchPairs) + '}'; - } - } - - static public class CountAggregationElement implements AggregationElement { - private final String resultColumn; - private final CountBySpecImpl underlyingStateFactory; - - public CountAggregationElement(String resultColumn) { - this.resultColumn = resultColumn; - underlyingStateFactory = new CountBySpecImpl(resultColumn); - } - - @Override - public AggregationSpec getSpec() { - return underlyingStateFactory; - } - - @Override - public String[] getSourceColumns() { - return new String[0]; - } - - @Override - public MatchPair[] getResultPairs() { - return new MatchPair[] {new MatchPair(resultColumn, resultColumn)}; - } - - @Override - public AggregationElement forRollup() { - return this; - } - - - @Override - public AggregationMemoKey getMemoKey() { - return getSpec().getMemoKey(); - } - - @Override - public String toString() { - return "Count(" + resultColumn + ")"; - } - } - - static public class NullAggregationElement implements AggregationElement { - private final Map> resultColumns; - private final NullAggregationSpecImpl underlyingStateFactory; - - NullAggregationElement(Map> resultColumns) { - this.resultColumns = resultColumns; - this.underlyingStateFactory = new NullAggregationSpecImpl(); - } - - @Override - public AggregationSpec getSpec() { - return underlyingStateFactory; - } - - @Override - public String[] getSourceColumns() { - return new String[0]; - } - - @Override - public MatchPair[] getResultPairs() { - return resultColumns.keySet().stream().map(rc -> new MatchPair(rc, rc)).toArray(MatchPair[]::new); - } - - @Override - public AggregationElement forRollup() { - throw new UnsupportedOperationException(); - } - - @Override - public AggregationMemoKey getMemoKey() { - return getSpec().getMemoKey(); - } - - @Override - public String toString() { - return "NullAggregationElement(" + resultColumns + ')'; - } - } - - static private class PartitionAggregationElement implements AggregationElement { - private final boolean leafLevel; - - private PartitionAggregationElement(boolean leafLevel) { - this.leafLevel = leafLevel; - } - - @Override - public AggregationSpec getSpec() { - throw new UnsupportedOperationException(); - } - - @Override - public String[] getSourceColumns() { - return new String[0]; - } - - @Override - public MatchPair[] getResultPairs() { - return new MatchPair[] {new MatchPair(RollupInfo.ROLLUP_COLUMN, RollupInfo.ROLLUP_COLUMN)}; - } - - @Override - public AggregationElement forRollup() { - return this; - } - - @Override - public AggregationMemoKey getMemoKey() { - // TODO: MEMOIZE! - return null; - } - - @Override - public String toString() { - return "PartitionAggregationElement(leafLevel=" + leafLevel + ')'; - } - - } - - public AggregationFactory(Collection aggregations) { - this(aggregations, false, false); - } - - public AggregationFactory(Collection aggregations, boolean isRollup, - boolean secondLevelRollup) { - this.isRollup = isRollup; - this.secondLevel = secondLevelRollup; - underlyingAggregations.addAll(aggregations); - - final Map> usedColumns = new LinkedHashMap<>(); - - for (final AggregationElement aggregationElement : underlyingAggregations) { - Stream.of(aggregationElement.getResultPairs()).map(MatchPair::leftColumn) - .forEach(rl -> usedColumns.computeIfAbsent(rl, x -> new ArrayList<>()).add(aggregationElement)); - } - - final Map> duplicates = - usedColumns.entrySet().stream().filter(kv -> kv.getValue().size() > 1) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - if (!duplicates.isEmpty()) { - final String errors = - duplicates.entrySet().stream().map(kv -> kv.getKey() + " used " + kv.getValue().size() + " times") - .collect(Collectors.joining(", ")); - throw new IllegalArgumentException("Duplicate output columns: " + errors); - } - } - - public AggregationFactory(AggregationElement... aggregations) { - this(Arrays.asList(aggregations), false, false); - } - - @Override - public AggregationMemoKey getMemoKey() { - final UnderlyingMemoKey[] underlyingMemoKeys = new UnderlyingMemoKey[underlyingAggregations.size()]; - for (int ii = 0; ii < underlyingMemoKeys.length; ++ii) { - final AggregationElement aggregationElement = underlyingAggregations.get(ii); - final AggregationMemoKey key = aggregationElement.getMemoKey(); - if (key == null) { - return null; - } - underlyingMemoKeys[ii] = new UnderlyingMemoKey(key, aggregationElement.getSourceColumns(), - aggregationElement.getResultPairs()); - } - - return new AggByMemoKey(underlyingMemoKeys); - } - - private static class UnderlyingMemoKey { - private final AggregationMemoKey componentMemoKey; - private final String[] sourceColumns; - private final MatchPair[] resultPairs; - - - private UnderlyingMemoKey(AggregationMemoKey componentMemoKey, String[] sourceColumns, - MatchPair[] resultPairs) { - this.componentMemoKey = componentMemoKey; - this.sourceColumns = sourceColumns; - this.resultPairs = resultPairs; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final UnderlyingMemoKey that = (UnderlyingMemoKey) o; - return Objects.equals(componentMemoKey, that.componentMemoKey) && - Arrays.equals(sourceColumns, that.sourceColumns) && - Arrays.equals(resultPairs, that.resultPairs); - } - - @Override - public int hashCode() { - int result = Objects.hash(componentMemoKey); - result = 31 * result + Arrays.hashCode(sourceColumns); - result = 31 * result + Arrays.hashCode(resultPairs); - return result; - } - } - - private static class AggByMemoKey implements AggregationMemoKey { - private final UnderlyingMemoKey[] underlyingMemoKeys; - - private AggByMemoKey(UnderlyingMemoKey[] underlyingMemoKeys) { - this.underlyingMemoKeys = underlyingMemoKeys; - } - - @Override - public int hashCode() { - return Arrays.hashCode(underlyingMemoKeys); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof AggByMemoKey - && Arrays.equals(underlyingMemoKeys, ((AggByMemoKey) obj).underlyingMemoKeys); - } - } - - @Override - public String toString() { - return "AggregationFactory{" + underlyingAggregations + '}'; - } - - public List getMatchPairs() { - return underlyingAggregations.stream().flatMap(c -> Arrays.stream(c.getResultPairs())) - .collect(Collectors.toList()); - } - - public AggregationContextFactory makeAggregationContextFactory() { - return (table, groupByColumns) -> { - final List operators = new ArrayList<>(); - final List inputNames = new ArrayList<>(); - final List> inputColumns = new ArrayList<>(); - final List transformers = new ArrayList<>(); - - int trackedFirstOrLastIndex = -1; - boolean externalFound = false; - - - for (final AggregationElement aggregationElement : underlyingAggregations) { - final boolean isStream = ((BaseTable) table).isStream(); - final boolean isAddOnly = ((BaseTable) table).isAddOnly(); - - if (aggregationElement instanceof CountAggregationElement) { - operators.add( - new CountAggregationOperator(((CountAggregationElement) aggregationElement).resultColumn)); - inputColumns.add(null); - inputNames.add(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); - } else if (aggregationElement instanceof AggregationElementImpl) { - final AggregationSpec inputAggregationSpec = aggregationElement.getSpec(); - - final boolean isNumeric = inputAggregationSpec.getClass() == SumSpec.class || - inputAggregationSpec.getClass() == AbsSumSpec.class || - inputAggregationSpec.getClass() == AvgSpec.class || - inputAggregationSpec.getClass() == VarSpec.class || - inputAggregationSpec.getClass() == StdSpec.class; - final boolean isCountDistinct = - inputAggregationSpec.getClass() == CountDistinctSpec.class; - final boolean isDistinct = inputAggregationSpec.getClass() == DistinctSpec.class; - final boolean isSelectDistinct = - inputAggregationSpec.getClass() == SelectDistinctSpecImpl.class; - final boolean isAggUnique = inputAggregationSpec.getClass() == UniqueSpec.class; - final boolean isMinMax = inputAggregationSpec instanceof MinMaxBySpecImpl; - final boolean isPercentile = - inputAggregationSpec.getClass() == PercentileBySpecImpl.class; - final boolean isSortedFirstOrLastBy = - inputAggregationSpec instanceof SortedFirstOrLastByFactoryImpl; - final boolean isFirst = inputAggregationSpec.getClass() == FirstBySpecImpl.class || - inputAggregationSpec.getClass() == TrackingFirstBySpecImpl.class || - (inputAggregationSpec.getClass() == KeyOnlyFirstOrLastBySpec.class && - !((KeyOnlyFirstOrLastBySpec) inputAggregationSpec).isLast()); - final boolean isLast = inputAggregationSpec.getClass() == LastBySpecImpl.class || - inputAggregationSpec.getClass() == TrackingLastBySpecImpl.class || - (inputAggregationSpec.getClass() == KeyOnlyFirstOrLastBySpec.class && - ((KeyOnlyFirstOrLastBySpec) inputAggregationSpec).isLast()); - final boolean isWeightedAverage = - inputAggregationSpec.getClass() == WeightedAverageSpecImpl.class; - final boolean isWeightedSum = - inputAggregationSpec.getClass() == WeightedSumSpecImpl.class; - final boolean isAggGroup = - inputAggregationSpec.getClass() == AggregationGroupSpec.class; - final boolean isFormula = - inputAggregationSpec.getClass() == AggregationFormulaSpec.class; - - // noinspection StatementWithEmptyBody - if (isSelectDistinct) { - // Select-distinct is accomplished as a side effect of aggregating on the group-by columns. - } else { - final MatchPair[] comboMatchPairs = ((AggregationElementImpl) aggregationElement).matchPairs; - if (isSortedFirstOrLastBy) { - // noinspection ConstantConditions - final SortedFirstOrLastByFactoryImpl sortedFirstOrLastByFactory = - (SortedFirstOrLastByFactoryImpl) inputAggregationSpec; - final boolean isSortedFirstBy = sortedFirstOrLastByFactory.isSortedFirst(); - - final MatchPair[] updatedMatchPairs; - if (sortedFirstOrLastByFactory.secondRollup - && sortedFirstOrLastByFactory.getSortColumnNames().length == 1 - && sortedFirstOrLastByFactory.getSortColumnNames()[0] - .endsWith(RollupConstants.ROLLUP_COLUMN_SUFFIX)) { - updatedMatchPairs = Arrays.copyOf(comboMatchPairs, comboMatchPairs.length + 1); - final String redirectionName = sortedFirstOrLastByFactory.getSortColumnNames()[0]; - updatedMatchPairs[updatedMatchPairs.length - 1] = - new MatchPair(redirectionName, redirectionName); - } else { - updatedMatchPairs = comboMatchPairs; - } - final AggregationContext sflac = SortedFirstOrLastByAggregationFactory - .getAggregationContext(table, sortedFirstOrLastByFactory.getSortColumnNames(), - isSortedFirstBy, true, updatedMatchPairs); - Assert.eq(sflac.operators.length, "sflac.operators.length", 1); - Assert.eq(sflac.inputColumns.length, "sflac.operators.length", 1); - Assert.eq(sflac.inputNames.length, "sflac.operators.length", 1); - operators.add(sflac.operators[0]); - inputColumns.add(sflac.inputColumns[0]); - inputNames.add(sflac.inputNames[0]); - } else if (isNumeric || isMinMax || isPercentile || isCountDistinct || isDistinct - || isAggUnique) { - // add the stuff - Arrays.stream(comboMatchPairs).forEach(mp -> { - if (isRollup && secondLevel) { - final boolean isAverage = - inputAggregationSpec.getClass() == AvgSpec.class; - final boolean isStd = - inputAggregationSpec.getClass() == StdSpec.class; - final boolean isVar = - inputAggregationSpec.getClass() == VarSpec.class; - final boolean isStdVar = isStd || isVar; - if (isAverage || isStdVar) { - final String runningSumName = - mp.leftColumn() + RollupConstants.ROLLUP_RUNNING_SUM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - final String runningSum2Name = - mp.leftColumn() + RollupConstants.ROLLUP_RUNNING_SUM2_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - final String nonNullName = - mp.leftColumn() + RollupConstants.ROLLUP_NONNULL_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - final String nanName = - mp.leftColumn() + RollupConstants.ROLLUP_NAN_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - final String picName = - mp.leftColumn() + RollupConstants.ROLLUP_PI_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - final String nicName = - mp.leftColumn() + RollupConstants.ROLLUP_NI_COUNT_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - - final boolean isFloatingPoint = table.hasColumns(nanName); - - // record non null count - inputColumns.add(table.getColumnSource(nonNullName)); - inputNames.add(new String[] {nonNullName}); - - // record running sum - inputColumns.add(table.getColumnSource(runningSumName)); - inputNames.add(new String[] {runningSumName}); - - if (isStdVar) { - inputColumns.add(table.getColumnSource(runningSum2Name)); - inputNames.add(new String[] {runningSum2Name}); - } - - if (isFloatingPoint) { - // record nans, positive and negative infinities - inputColumns.add(table.getColumnSource(nanName)); - inputNames.add(new String[] {nanName}); - inputColumns.add(table.getColumnSource(picName)); - inputNames.add(new String[] {picName}); - inputColumns.add(table.getColumnSource(nicName)); - inputNames.add(new String[] {nicName}); - } - - // then the input column for the updater (reavg/revar) operator - inputColumns.add(null); - - // now add add the operators, and the final inputNames that matches the updating - // operator - final LongChunkedSumOperator nonNull = - new LongChunkedSumOperator(false, nonNullName); - operators.add(nonNull); - - if (isFloatingPoint) { - final DoubleChunkedSumOperator runningSum = - new DoubleChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - - final DoubleChunkedSumOperator runningSum2; - if (isStdVar) { - runningSum2 = new DoubleChunkedSumOperator(false, runningSum2Name); - operators.add(runningSum2); - } else { - runningSum2 = null; - } - - final LongChunkedSumOperator nanSum = - new LongChunkedSumOperator(false, nanName); - operators.add(nanSum); - final LongChunkedSumOperator picSum = - new LongChunkedSumOperator(false, picName); - operators.add(picSum); - final LongChunkedSumOperator nicSum = - new LongChunkedSumOperator(false, nicName); - operators.add(nicSum); - - if (isAverage) { - if (table.getColumnSource(mp.leftColumn()) - .getChunkType() == ChunkType.Float) { - operators.add( - new FloatChunkedReAvgOperator(mp.leftColumn(), runningSum, - nonNull, nanSum, picSum, nicSum)); - } else if (table.getColumnSource(mp.leftColumn()) - .getChunkType() == ChunkType.Double) { - operators.add( - new DoubleChunkedReAvgOperator(mp.leftColumn(), runningSum, - nonNull, nanSum, picSum, nicSum)); - } else { - throw new UnsupportedOperationException(); - } - } else { - if (table.getColumnSource(mp.leftColumn()) - .getChunkType() == ChunkType.Float - || table.getColumnSource(mp.leftColumn()) - .getChunkType() == ChunkType.Double) { - operators.add(new FloatChunkedReVarOperator(mp.leftColumn(), isStd, - runningSum, runningSum2, nonNull, nanSum, picSum, nicSum)); - } else { - throw new UnsupportedOperationException(); - } - } - - // our final operator is updated if any input changes - final String[] inputNamesForColumn = new String[isStdVar ? 6 : 5]; - inputNamesForColumn[0] = nonNullName; - inputNamesForColumn[1] = runningSumName; - inputNamesForColumn[2] = nanName; - inputNamesForColumn[3] = picName; - inputNamesForColumn[4] = nicName; - if (isStdVar) { - inputNamesForColumn[5] = runningSum2Name; - } - inputNames.add(inputNamesForColumn); - } else if (isStdVar) { - final boolean isBigInteger = BigInteger.class - .isAssignableFrom(table.getColumnSource(runningSumName).getType()); - final boolean isBigDecimal = BigDecimal.class - .isAssignableFrom(table.getColumnSource(runningSumName).getType()); - - if (isBigInteger) { - final BigIntegerChunkedSumOperator runningSum = - new BigIntegerChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - final BigIntegerChunkedSumOperator runningSum2 = - new BigIntegerChunkedSumOperator(false, runningSum2Name); - operators.add(runningSum2); - operators.add(new BigIntegerChunkedReVarOperator(mp.leftColumn(), isStd, - runningSum, runningSum2, nonNull)); - } else if (isBigDecimal) { - final BigDecimalChunkedSumOperator runningSum = - new BigDecimalChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - final BigDecimalChunkedSumOperator runningSum2 = - new BigDecimalChunkedSumOperator(false, runningSum2Name); - operators.add(runningSum2); - operators.add(new BigDecimalChunkedReVarOperator(mp.leftColumn(), isStd, - runningSum, runningSum2, nonNull)); - } else { - final DoubleChunkedSumOperator runningSum = - new DoubleChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - final DoubleChunkedSumOperator runningSum2 = - new DoubleChunkedSumOperator(false, runningSum2Name); - operators.add(runningSum2); - operators.add(new IntegralChunkedReVarOperator(mp.leftColumn(), isStd, - runningSum, runningSum2, nonNull)); - } - // our final operator is updated if any input changes - inputNames.add(new String[] {nonNullName, runningSumName, runningSum2Name}); - } else { // is an average and not floating point - final boolean isBigDecimal = BigDecimal.class - .isAssignableFrom(table.getColumnSource(runningSumName).getType()); - final boolean isBigInteger = BigInteger.class - .isAssignableFrom(table.getColumnSource(runningSumName).getType()); - - if (isBigInteger) { - final BigIntegerChunkedSumOperator runningSum = - new BigIntegerChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - operators.add( - new BigIntegerChunkedReAvgOperator(mp.leftColumn(), runningSum, - nonNull)); - } else if (isBigDecimal) { - final BigDecimalChunkedSumOperator runningSum = - new BigDecimalChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - operators.add( - new BigDecimalChunkedReAvgOperator(mp.leftColumn(), runningSum, - nonNull)); - } else { - final LongChunkedSumOperator runningSum = - new LongChunkedSumOperator(false, runningSumName); - operators.add(runningSum); - operators.add( - new IntegralChunkedReAvgOperator(mp.leftColumn(), runningSum, - nonNull)); - } - - // our final operator is updated if any input changes - inputNames.add(new String[] {nonNullName, runningSumName}); - } - return; - } else if (isCountDistinct || isDistinct || isAggUnique) { - final String ssmColName = - mp.leftColumn() + RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID + RollupConstants.ROLLUP_COLUMN_SUFFIX; - final ColumnSource> ssmSource = - table.getColumnSource(ssmColName); - final ColumnSource lastLevelResult = table.getColumnSource(mp.leftColumn()); - final boolean countNulls; - final IterativeChunkedAggregationOperator op; - if (isDistinct) { - // noinspection ConstantConditions - countNulls = - ((DistinctSpec) inputAggregationSpec).countNulls(); - op = IterativeOperatorSpec.getDistinctChunked( - lastLevelResult.getComponentType(), mp.leftColumn(), countNulls, - true, - true); - } else if (isCountDistinct) { - // noinspection ConstantConditions - countNulls = ((CountDistinctSpec) inputAggregationSpec) - .countNulls(); - op = IterativeOperatorSpec.getCountDistinctChunked( - ssmSource.getComponentType(), mp.leftColumn(), countNulls, true, - true); - } else { - // noinspection ConstantConditions - countNulls = - ((UniqueSpec) inputAggregationSpec).countNulls(); - // noinspection ConstantConditions - op = IterativeOperatorSpec.getUniqueChunked( - lastLevelResult.getType(), mp.leftColumn(), countNulls, ((UniqueSpec) inputAggregationSpec).getNoKeyValue(), ((UniqueSpec) inputAggregationSpec) - .getNonUniqueValue(), true, - true); - } - - inputColumns.add(ssmSource); - inputNames.add(new String[] {ssmColName}); - operators.add(op); - - return; - } - } - - final ColumnSource columnSource = table.getColumnSource(mp.rightColumn()); - final Class type = columnSource.getType(); - final ColumnSource inputSource = columnSource.getType() == DateTime.class - ? ReinterpretUtils.dateTimeToLongSource(columnSource) - : columnSource; - - final String resultName = mp.leftColumn(); - final boolean hasSource; - if (isMinMax) { - final boolean isMinimum = - ((MinMaxBySpecImpl) inputAggregationSpec).isMinimum(); - final OptionalInt priorMinMax = IntStream.range(0, inputColumns.size()) - .filter(idx -> (inputColumns.get(idx) == inputSource) - && (operators.get(idx) instanceof SsmChunkedMinMaxOperator)) - .findFirst(); - if (priorMinMax.isPresent()) { - final SsmChunkedMinMaxOperator ssmChunkedMinMaxOperator = - (SsmChunkedMinMaxOperator) operators.get(priorMinMax.getAsInt()); - operators.add( - ssmChunkedMinMaxOperator.makeSecondaryOperator(isMinimum, resultName)); - hasSource = false; - } else { - operators.add(IterativeOperatorSpec.getMinMaxChunked(type, resultName, isMinimum, - isStream || isAddOnly)); - hasSource = true; - } - } else if (isPercentile) { - if (isRollup) { - throw new UnsupportedOperationException( - "Percentile or Median can not be used in a rollup!"); - } - operators.add(IterativeOperatorSpec.getPercentileChunked(type, - resultName, ((PercentileBySpecImpl) inputAggregationSpec) - .getPercentile(), - ((PercentileBySpecImpl) inputAggregationSpec) - .getAverageMedian() - )); - hasSource = true; - } else { - operators.add(((IterativeOperatorSpec) inputAggregationSpec) - .getChunkedOperator(type, resultName, isRollup)); - hasSource = true; - } - - if (hasSource) { - inputColumns.add(inputSource); - } else { - inputColumns.add(null); - } - inputNames.add(new String[] {mp.rightColumn()}); - }); - } else if (isFirst || isLast) { - inputColumns.add(null); - final String exposeRedirectionAs; - if (isRollup) { - exposeRedirectionAs = - makeRedirectionName((IterativeIndexSpec) inputAggregationSpec); - } else if (inputAggregationSpec instanceof KeyOnlyFirstOrLastBySpec) { - exposeRedirectionAs = ((KeyOnlyFirstOrLastBySpec) inputAggregationSpec) - .getResultColumn(); - } else { - exposeRedirectionAs = null; - } - - if (table.isRefreshing()) { - if (isStream) { - operators.add(isFirst ? new StreamFirstChunkedOperator(comboMatchPairs, table) - : new StreamLastChunkedOperator(comboMatchPairs, table)); - } else if (isAddOnly) { - operators.add(new AddOnlyFirstOrLastChunkedOperator(isFirst, comboMatchPairs, table, - exposeRedirectionAs)); - } else { - if (trackedFirstOrLastIndex >= 0) { - final IterativeChunkedAggregationOperator operator = - operators.get(trackedFirstOrLastIndex); - final FirstOrLastChunkedOperator firstOrLastChunkedOperator = - (FirstOrLastChunkedOperator) operator; - operators.add(firstOrLastChunkedOperator.makeSecondaryOperator(isFirst, - comboMatchPairs, table, exposeRedirectionAs)); - } else { - operators.add(new FirstOrLastChunkedOperator(isFirst, comboMatchPairs, table, - exposeRedirectionAs)); - trackedFirstOrLastIndex = operators.size() - 1; - } - } - } else { - operators.add(new StaticFirstOrLastChunkedOperator(isFirst, comboMatchPairs, table, - exposeRedirectionAs)); - } - inputNames.add(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); - } else if (isAggGroup) { - if (isStream) { - throw streamUnsupported("AggGroup"); - } - inputColumns.add(null); - operators.add(new GroupByChunkedOperator((QueryTable) table, true, comboMatchPairs)); - inputNames.add(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); - } else if (isFormula) { - if (isStream) { - throw streamUnsupported("AggFormula"); - } - final AggregationFormulaSpec formulaStateFactory = - (AggregationFormulaSpec) inputAggregationSpec; - final GroupByChunkedOperator groupByChunkedOperator = - new GroupByChunkedOperator((QueryTable) table, false, - Arrays.stream(comboMatchPairs).map(MatchPair::rightColumn) - .map(MatchPairFactory::getExpression).toArray(MatchPair[]::new)); - final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator( - groupByChunkedOperator, true, formulaStateFactory.getFormula(), - formulaStateFactory.getColumnParamName(), comboMatchPairs); - inputColumns.add(null); - operators.add(formulaChunkedOperator); - inputNames.add(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); - } else if (isWeightedAverage || isWeightedSum) { - final String weightName = isWeightedAverage - ? ((WeightedAverageSpecImpl) inputAggregationSpec).getWeightName() - : ((WeightedSumSpecImpl) inputAggregationSpec).getWeightName(); - WeightedAverageSumAggregationFactory.getOperatorsAndInputs(table, - weightName, isWeightedSum, comboMatchPairs, operators, inputNames, inputColumns); - } else { - throw new UnsupportedOperationException( - "Unknown AggregationElementImpl: " + inputAggregationSpec.getClass()); - } - } - } else if (aggregationElement instanceof NullAggregationElement) { - transformers.add(new NullColumnAggregationTransformer( - ((NullAggregationElement) aggregationElement).resultColumns)); - } else if (aggregationElement instanceof PartitionAggregationElement) { - if (!isRollup) { - throw new IllegalStateException("PartitionAggregationElement must be used only with rollups."); - } - inputColumns.add(null); - inputNames.add(CollectionUtil.ZERO_LENGTH_STRING_ARRAY); - final boolean includeConstituents = ((PartitionAggregationElement) aggregationElement).leafLevel; - if (includeConstituents) { - if (isStream) { - throw streamUnsupported("rollup with included constituents"); - } - Assert.eqFalse(secondLevel, "secondLevel"); - } - - final QueryTable parentTable = (QueryTable) table; - final QueryTable adjustedTable; - final List columnsToDrop = - parentTable.getDefinition().getColumnStream().map(ColumnDefinition::getName) - .filter(cn -> cn.endsWith(RollupConstants.ROLLUP_COLUMN_SUFFIX)).collect(Collectors.toList()); - if (!columnsToDrop.isEmpty()) { - adjustedTable = (QueryTable) parentTable.dropColumns(columnsToDrop); - } else { - if (includeConstituents) { - adjustedTable = (QueryTable) parentTable.updateView(RollupInfo.ROLLUP_COLUMN + "=" + null); - } else { - adjustedTable = parentTable; - } - } - if (adjustedTable != parentTable && parentTable.hasAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE)) { - adjustedTable.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, - parentTable.getAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE)); - } - final PartitionByChunkedOperator.AttributeCopier copier; - if (includeConstituents) { - copier = RollupAttributeCopier.LEAF_WITHCONSTITUENTS_INSTANCE; - } else { - copier = RollupAttributeCopier.DEFAULT_INSTANCE; - } - final PartitionByChunkedOperator tableMapOperator = new PartitionByChunkedOperator(parentTable, - adjustedTable, copier, Collections.emptyList(), groupByColumns); - operators.add(tableMapOperator); - - if (groupByColumns.length == 0) { - transformers.add(new StaticColumnSourceTransformer(RollupInfo.ROLLUP_COLUMN, - new SingleValueObjectColumnSource<>(SmartKey.EMPTY))); - } else if (groupByColumns.length == 1) { - transformers.add(new RollupKeyColumnDuplicationTransformer(groupByColumns[0])); - } else { - transformers.add(new RollupSmartKeyColumnDuplicationTransformer(groupByColumns)); - } - - transformers.add(new RollupTableMapAndReverseLookupAttributeSetter(tableMapOperator, this, - secondLevel, includeConstituents)); - - externalFound = true; - } else { - throw new UnsupportedOperationException( - "Unknown AggregationElement: " + aggregationElement.getClass()); - } - } - - if (!externalFound && isRollup && !secondLevel) { - transformers.add(new NoKeyLeafRollupAttributeSetter()); - } - - final IterativeChunkedAggregationOperator[] operatorsArray = operators.toArray( - IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY); - final AggregationContextTransformer[] transformersArray = transformers - .toArray(AggregationContextTransformer.ZERO_LENGTH_AGGREGATION_CONTEXT_TRANSFORMER_ARRAY); - final String[][] inputNamesArray = inputNames.toArray(CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY); - // noinspection unchecked - final ChunkSource.WithPrev[] inputColumnsArray = - inputColumns.toArray(ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); - - return new AggregationContext(operatorsArray, inputNamesArray, inputColumnsArray, transformersArray); - }; - } - - private static UnsupportedOperationException streamUnsupported(@NotNull final String operatorTypeName) { - return new UnsupportedOperationException("Stream tables do not support " + operatorTypeName - + "; use StreamTableTools.streamToAppendOnlyTable to accumulate full history"); - } - - @NotNull - private static String makeRedirectionName(IterativeIndexSpec inputAggregationStateFactory) { - return IterativeIndexSpec.ROW_REDIRECTION_PREFIX + inputAggregationStateFactory.rollupColumnIdentifier - + RollupConstants.ROLLUP_COLUMN_SUFFIX; - } - - private static class RollupTableMapAndReverseLookupAttributeSetter implements AggregationContextTransformer { - - private final PartitionByChunkedOperator tableMapOperator; - private final AggregationFactory factory; - private final boolean secondLevel; - private final boolean includeConstituents; - private ReverseLookup reverseLookup; - - RollupTableMapAndReverseLookupAttributeSetter(PartitionByChunkedOperator tableMapOperator, - AggregationFactory factory, boolean secondLevel, boolean includeConstituents) { - this.tableMapOperator = tableMapOperator; - this.factory = factory; - this.secondLevel = secondLevel; - this.includeConstituents = includeConstituents; - } - - @Override - public QueryTable transformResult(QueryTable table) { - table.setAttribute(QueryTable.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, tableMapOperator.getTableMap()); - if (secondLevel || includeConstituents) { - table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, reverseLookup); - } else { - setLeafRollupAttributes(table); - } - return table; - } - - @Override - public void setReverseLookupFunction(ToIntFunction reverseLookup) { - this.reverseLookup = new ReverseLookupAdapter(reverseLookup); - } - - private class ReverseLookupAdapter implements ReverseLookup { - private final ToIntFunction reverseLookup; - - public ReverseLookupAdapter(ToIntFunction reverseLookup) { - this.reverseLookup = reverseLookup; - } - - @Override - public long get(Object key) { - return reverseLookup.applyAsInt(key); - } - - @Override - public long getPrev(Object key) { - return get(key); - } - - @Override - public long getNoEntryValue() { - return -1; - } - - @Override - public String[] getKeyColumns() { - return factory.getKeyColumns(); - } - } - } - - private static class NoKeyLeafRollupAttributeSetter implements AggregationContextTransformer { - @Override - public QueryTable transformResult(QueryTable table) { - setLeafRollupAttributes(table); - return table; - } - } - - private static void setLeafRollupAttributes(QueryTable table) { - table.setAttribute(Table.ROLLUP_LEAF_ATTRIBUTE, RollupInfo.LeafType.Normal); - table.setAttribute(Table.HIERARCHICAL_CHILDREN_TABLE_MAP_ATTRIBUTE, EmptyTableMap.INSTANCE); - table.setAttribute(Table.REVERSE_LOOKUP_ATTRIBUTE, ReverseLookup.NULL); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.java deleted file mode 100644 index b0d8294d38d..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import java.util.*; - -public class AggregationFormulaSpec extends AggregationGroupSpec { - - private final String formula; - private final String columnParamName; - - public AggregationFormulaSpec(String formula, String columnParamName) { - this.formula = formula; - this.columnParamName = columnParamName; - } - - public String getFormula() { - return formula; - } - - public String getColumnParamName() { - return columnParamName; - } - - private static class MemoKey implements AggregationMemoKey { - private final String formula; - private final String columnParamName; - - private MemoKey(String formula, String columnParamName) { - this.formula = formula; - this.columnParamName = columnParamName; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final MemoKey memoKey = (MemoKey) o; - return Objects.equals(formula, memoKey.formula) && - Objects.equals(columnParamName, memoKey.columnParamName); - } - - @Override - public int hashCode() { - return Objects.hash(formula, columnParamName); - } - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(formula, columnParamName); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationGroupSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationGroupSpec.java deleted file mode 100644 index d47963565dd..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationGroupSpec.java +++ /dev/null @@ -1,14 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public class AggregationGroupSpec implements AggregationSpec { - private static final AggregationMemoKey AGGREGATION_INDEX_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return AGGREGATION_INDEX_INSTANCE; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationMemoKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationMemoKey.java deleted file mode 100644 index 0cfbdc99a99..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationMemoKey.java +++ /dev/null @@ -1,7 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -/** - * Marker interface for AggregationStates to use for memoization keys. - */ -public interface AggregationMemoKey { -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpec.java deleted file mode 100644 index 4f8a7291f28..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpec.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.table.Table; - -/** - * AggregationSpecs are passed to {@link Table#groupBy} operation, and used to supply operation-specific instructions. - */ -public interface AggregationSpec { - /** - * Produces a MemoKey for this AggregationSpec. - * - *

- * If two AggregationSpecs have equal memoKeys, then {@link Table#groupBy} operations that have the same group by - * columns may be memoized. In that case instead of recomputing the result; the original result will be used. - *

- * - *

- * If null is returned, the operation will not be memoized. - *

- * - * @return an AggregationMemoKey, null if this operation can not be memoized. - */ - default AggregationMemoKey getMemoKey() { - return null; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java deleted file mode 100644 index 76087861af9..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.java +++ /dev/null @@ -1,146 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.api.ColumnName; -import io.deephaven.api.SortColumn; -import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.api.agg.spec.AggSpecAbsSum; -import io.deephaven.api.agg.spec.AggSpecAvg; -import io.deephaven.api.agg.spec.AggSpecCountDistinct; -import io.deephaven.api.agg.spec.AggSpecDistinct; -import io.deephaven.api.agg.spec.AggSpecFirst; -import io.deephaven.api.agg.spec.AggSpecFormula; -import io.deephaven.api.agg.spec.AggSpecGroup; -import io.deephaven.api.agg.spec.AggSpecLast; -import io.deephaven.api.agg.spec.AggSpecMax; -import io.deephaven.api.agg.spec.AggSpecMedian; -import io.deephaven.api.agg.spec.AggSpecMin; -import io.deephaven.api.agg.spec.AggSpecPercentile; -import io.deephaven.api.agg.spec.AggSpecSortedFirst; -import io.deephaven.api.agg.spec.AggSpecSortedLast; -import io.deephaven.api.agg.spec.AggSpecStd; -import io.deephaven.api.agg.spec.AggSpecSum; -import io.deephaven.api.agg.spec.AggSpecUnique; -import io.deephaven.api.agg.spec.AggSpecVar; -import io.deephaven.api.agg.spec.AggSpecWAvg; -import io.deephaven.api.agg.spec.AggSpecWSum; - -import java.util.Objects; - -import static io.deephaven.engine.table.impl.QueryTable.TRACKED_FIRST_BY; -import static io.deephaven.engine.table.impl.QueryTable.TRACKED_LAST_BY; - -public class AggregationSpecAdapter implements AggSpec.Visitor { - - public static AggregationSpec of(AggSpec spec) { - return spec.walk(new AggregationSpecAdapter()).out(); - } - - private AggregationSpec out; - - - public AggregationSpec out() { - return Objects.requireNonNull(out); - } - - @Override - public void visit(AggSpecAbsSum absSum) { - out = new AbsSumSpec(); - } - - @Override - public void visit(AggSpecCountDistinct countDistinct) { - out = new CountDistinctSpec(countDistinct.countNulls()); - } - - @Override - public void visit(AggSpecDistinct distinct) { - out = new DistinctSpec(distinct.includeNulls()); - } - - @Override - public void visit(AggSpecGroup group) { - out = new AggregationGroupSpec(); - } - - @Override - public void visit(AggSpecAvg avg) { - out = new AvgSpec(); - } - - @Override - public void visit(AggSpecFirst first) { - out = TRACKED_FIRST_BY ? new TrackingFirstBySpecImpl() : new FirstBySpecImpl(); - } - - @Override - public void visit(AggSpecFormula formula) { - out = new AggregationFormulaSpec(formula.formula(), formula.formulaParam()); - } - - @Override - public void visit(AggSpecLast last) { - out = TRACKED_LAST_BY ? new TrackingLastBySpecImpl() : new LastBySpecImpl(); - } - - @Override - public void visit(AggSpecMax max) { - out = new MinMaxBySpecImpl(false); - } - - @Override - public void visit(AggSpecMedian median) { - out = new PercentileBySpecImpl(0.50d, median.averageMedian()); - } - - @Override - public void visit(AggSpecMin min) { - out = new MinMaxBySpecImpl(true); - } - - @Override - public void visit(AggSpecPercentile pct) { - out = new PercentileBySpecImpl(pct.percentile(), pct.averageMedian()); - } - - @Override - public void visit(AggSpecSortedFirst sortedFirst) { - out = new SortedFirstBy( - sortedFirst.columns().stream().map(SortColumn::column).map(ColumnName::name).toArray(String[]::new)); - } - - @Override - public void visit(AggSpecSortedLast sortedLast) { - out = new SortedLastBy( - sortedLast.columns().stream().map(SortColumn::column).map(ColumnName::name).toArray(String[]::new)); - } - - @Override - public void visit(AggSpecStd std) { - out = new StdSpec(); - } - - @Override - public void visit(AggSpecSum sum) { - out = new SumSpec(); - } - - @Override - public void visit(AggSpecUnique unique) { - out = new UniqueSpec(unique.includeNulls(), null, unique.nonUniqueSentinel()); - } - - @Override - public void visit(AggSpecWAvg wAvg) { - out = new WeightedAverageSpecImpl(wAvg.weight().name()); - } - - @Override - public void visit(AggSpecWSum wSum) { - out = new WeightedSumSpecImpl(wSum.weight().name()); - } - - @Override - public void visit(AggSpecVar var) { - out = new VarSpec(); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgSpec.java deleted file mode 100644 index ed5e62baed6..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgSpec.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * Factory for iterative average aggregations. - */ -public class AvgSpec extends IterativeOperatorSpec { - public AvgSpec() {} - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getAvgChunked(type, name, exposeInternalColumns); - } - - private static final AggregationMemoKey AVG_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return AVG_INSTANCE; - } - - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return this; - } - - @Override - public String toString() { - return "Avg"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgState.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgState.java deleted file mode 100644 index 2cead08d956..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgState.java +++ /dev/null @@ -1,60 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TableToolsShowControl; -import io.deephaven.engine.updategraph.LogicalClock; -import org.jetbrains.annotations.NotNull; - -@TableToolsShowControl(getWidth = 40) -class AvgState implements PreviousStateProvider { - AvgState(boolean previous) { - if (!previous) { - prevValue = createPrev(); - changeTime = -1; - } else { - prevValue = null; - changeTime = -1; - } - } - - @NotNull - AvgState createPrev() { - return new AvgState(true); - } - - // only used in the current state - private long changeTime; - private final AvgState prevValue; - - protected double runningSum; - protected long nonNullCount; - - double currentValue() { - return runningSum / nonNullCount; - } - - @Override - public AvgState prev() { - return prevValue; - } - - void checkUpdates() { - final long currentStep = LogicalClock.DEFAULT.currentStep(); - if (changeTime != currentStep) { - savePrevious(); - changeTime = currentStep; - } - } - - void savePrevious() { - prev().runningSum = runningSum; - prev().nonNullCount = nonNullCount; - } - - @Override - public String toString() { - return "Avg{" + - "sum=" + runningSum + - ", nonNull=" + nonNullCount + - '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgStateWithNan.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgStateWithNan.java deleted file mode 100644 index ff5e4f6add1..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AvgStateWithNan.java +++ /dev/null @@ -1,41 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TableToolsShowControl; -import org.jetbrains.annotations.NotNull; - -@TableToolsShowControl(getWidth = 40) -class AvgStateWithNan extends AvgState { - protected long nanCount; - - AvgStateWithNan(boolean previous) { - super(previous); - } - - @NotNull - @Override - AvgStateWithNan createPrev() { - return new AvgStateWithNan(true); - } - - double currentValue() { - if (nanCount > 0) { - return Double.NaN; - } - return runningSum / nonNullCount; - } - - @Override - void savePrevious() { - super.savePrevious(); - ((AvgStateWithNan) prev()).nanCount = nanCount; - } - - @Override - public String toString() { - return "Avg{" + - "sum=" + runningSum + - ", nonNull=" + nonNullCount + - ", nan=" + nanCount + - '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java deleted file mode 100644 index 18f59a90b1b..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/BaseConverter.java +++ /dev/null @@ -1,19 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.Pair; -import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.BaseTable; -import org.jetbrains.annotations.NotNull; - -import java.util.ArrayList; -import java.util.List; - -/** - * @author rcaudy - * @since 2022-01-21 - */ - diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountAggregationFactory.java deleted file mode 100644 index e1693aafd33..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountAggregationFactory.java +++ /dev/null @@ -1,36 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.Table; -import io.deephaven.api.util.NameValidator; -import io.deephaven.engine.table.ChunkSource; -import org.jetbrains.annotations.NotNull; - -public class CountAggregationFactory implements AggregationContextFactory { - private final String resultName; - private static final ChunkSource.WithPrev[] nullSourceArray = {null}; - - public CountAggregationFactory(final String resultName) { - this.resultName = NameValidator.validateColumnName(resultName); - } - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - final IterativeChunkedAggregationOperator[] countOperator = new IterativeChunkedAggregationOperator[1]; - countOperator[0] = new CountAggregationOperator(resultName); - - final String[][] inputNameArray = new String[1][0]; - inputNameArray[0] = CollectionUtil.ZERO_LENGTH_STRING_ARRAY; - - // noinspection unchecked - return new AggregationContext(countOperator, - inputNameArray, - nullSourceArray); - } - - @Override - public String toString() { - return "Count"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountBySpecImpl.java deleted file mode 100644 index ea527519568..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountBySpecImpl.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import java.util.Objects; - -/** - * Aggregation state factory for countBy() operation. - */ -public class CountBySpecImpl extends ReaggregatableStatefactory { - private final String countColumnName; - - public CountBySpecImpl(String countColumnName) { - this.countColumnName = countColumnName; - } - - public String getCountName() { - return countColumnName; - } - - private static class MemoKey implements AggregationMemoKey { - private final String countColumnName; - - private MemoKey(String countColumnName) { - this.countColumnName = countColumnName; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final MemoKey memoKey = (MemoKey) o; - return Objects.equals(countColumnName, memoKey.countColumnName); - } - - @Override - public int hashCode() { - return Objects.hash(countColumnName); - } - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(countColumnName); - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return new SumSpec(); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountDistinctSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountDistinctSpec.java deleted file mode 100644 index cd559b9bcbe..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountDistinctSpec.java +++ /dev/null @@ -1,56 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -/** - * An Iterative state factory that computes the count of distinct items within a particular state. - */ -public class CountDistinctSpec extends IterativeOperatorSpec { - private static final AggregationMemoKey NO_NULLS_INSTANCE = new AggregationMemoKey() {}; - private static final AggregationMemoKey WITH_NULLS_INSTANCE = new AggregationMemoKey() {}; - private final boolean secondRollup; - private final boolean countNulls; - - CountDistinctSpec() { - this(false, false); - } - - CountDistinctSpec(boolean countNulls) { - this(false, countNulls); - } - - private CountDistinctSpec(boolean secondRollup, boolean countNulls) { - this.secondRollup = secondRollup; - this.countNulls = countNulls; - } - - @Override - public AggregationMemoKey getMemoKey() { - return countNulls ? WITH_NULLS_INSTANCE : NO_NULLS_INSTANCE; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - CountDistinctSpec forRollup() { - return this; - } - - @Override - CountDistinctSpec rollupFactory() { - return new CountDistinctSpec(true, countNulls); - } - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getCountDistinctChunked(type, name, countNulls, exposeInternalColumns, secondRollup); - } - - // endregion - - public boolean countNulls() { - return countNulls; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DistinctSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DistinctSpec.java deleted file mode 100644 index af9e326f2ea..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DistinctSpec.java +++ /dev/null @@ -1,57 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -/** - * An Iterative state factory that computes the count of distinct items within a particular state. It supports rollups, - * and converts itself into a Sum at the second level of the rollup. - */ -public class DistinctSpec extends IterativeOperatorSpec { - private static final AggregationMemoKey NO_NULLS_INSTANCE = new AggregationMemoKey() {}; - private static final AggregationMemoKey WITH_NULLS_INSTANCE = new AggregationMemoKey() {}; - private final boolean secondRollup; - private final boolean countNulls; - - DistinctSpec() { - this(false, false); - } - - DistinctSpec(boolean countNulls) { - this(false, countNulls); - } - - private DistinctSpec(boolean secondRollup, boolean countNulls) { - this.secondRollup = secondRollup; - this.countNulls = countNulls; - } - - @Override - public AggregationMemoKey getMemoKey() { - return countNulls ? WITH_NULLS_INSTANCE : NO_NULLS_INSTANCE; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - DistinctSpec forRollup() { - return this; - } - - @Override - DistinctSpec rollupFactory() { - return new DistinctSpec(true, countNulls); - } - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getDistinctChunked(type, name, countNulls, exposeInternalColumns, secondRollup); - } - - // endregion - - public boolean countNulls() { - return countNulls; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java deleted file mode 100644 index 6ee100c8adf..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstBySpecImpl.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * State factory for firstBy using an InterativeIndexState to create a RowRedirection. - */ -public class FirstBySpecImpl extends IterativeIndexSpec { - public FirstBySpecImpl() { - this(false, false, 0); - } - - private FirstBySpecImpl(boolean lowestRollup, boolean secondRollup, int rollupColumnIdentifier) { - super(lowestRollup, secondRollup, rollupColumnIdentifier); - } - - private static final AggregationMemoKey FIRST_BY_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return FIRST_BY_INSTANCE; - } - - @Override - ReaggregatableStatefactory forRollup() { - return new FirstBySpecImpl(true, false, 0); - } - - /** - * Sort the results by the original row key when aggregating on state. - */ - @Override - ReaggregatableStatefactory rollupFactory() { - return new SortedFirstOrLastByFactoryImpl(true, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); - } - - @Override - public String toString() { - if (!lowestRollup && !secondRollup) { - return "FirstByStateFactory"; - } else { - return "FirstByStateFactory{" + - "lowestRollup=" + lowestRollup + - ", secondRollup=" + secondRollup + - ", rollupColumnIdentifier=" + rollupColumnIdentifier + - '}'; - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java deleted file mode 100644 index 59594407d66..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeIndexSpec.java +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.AbstractColumnSource; -import io.deephaven.engine.table.impl.MutableColumnSourceGetDefaults; -import io.deephaven.engine.table.impl.util.WritableRowRedirection; -import io.deephaven.engine.table.impl.util.RowRedirection; -import org.jetbrains.annotations.NotNull; - -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; - -import static io.deephaven.engine.table.impl.by.RollupConstants.*; - -public abstract class IterativeIndexSpec extends ReaggregatableStatefactory { - static final String ROW_REDIRECTION_PREFIX = "RowRedirection_"; - private final Map nameToDestColumns = new LinkedHashMap<>(); - final protected RowRedirection rowRedirection = WritableRowRedirection.FACTORY.createRowRedirection(8); - private boolean firstTime; - final boolean lowestRollup; - final boolean secondRollup; - final int rollupColumnIdentifier; - - // if we have multiple aggregations, we do not want them to have conflicting column names, so we use an identifier - // to find them - private final static AtomicInteger nextRollupColumnIdentifier = new AtomicInteger(1); - - IterativeIndexSpec(boolean lowestRollup, boolean secondRollup, int rollupColumnIdentifier) { - firstTime = true; - this.lowestRollup = lowestRollup; - this.secondRollup = secondRollup; - if (lowestRollup) { - this.rollupColumnIdentifier = nextRollupColumnIdentifier.getAndIncrement(); - } else { - this.rollupColumnIdentifier = rollupColumnIdentifier; - } - } - - @NotNull - private String getRedirectionName() { - return ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + ROLLUP_COLUMN_SUFFIX; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - throw new UnsupportedOperationException(); - } - - @Override - ReaggregatableStatefactory rollupFactory() { - throw new UnsupportedOperationException(); - } - - class RedirectionValueColumnSource extends AbstractColumnSource - implements MutableColumnSourceGetDefaults.ForLong { - RedirectionValueColumnSource() { - super(Long.class); - } - - @Override - public void startTrackingPrevValues() { - // Nothing to do. - } - - @Override - public long getLong(long index) { - return rowRedirection.get(index); - } - - @Override - public long getPrevLong(long index) { - return rowRedirection.getPrev(index); - } - - @Override - public boolean isImmutable() { - return false; - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java deleted file mode 100644 index 4c2053afbb3..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.java +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.time.DateTime; -import io.deephaven.engine.table.impl.by.ssmcountdistinct.DistinctOperatorFactory; -import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; -import io.deephaven.engine.table.impl.by.ssmpercentile.SsmChunkedPercentileOperator; - -import java.math.BigDecimal; -import java.math.BigInteger; - -/** - * Creates iterative operators for the supplied type. - */ -public abstract class IterativeOperatorSpec extends ReaggregatableStatefactory - implements IterativeChunkedOperatorFactory { - - IterativeOperatorSpec() {} - - @Override - public abstract IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns); - - static IterativeChunkedAggregationOperator getSumChunked(Class type, String name) { - if (type == Boolean.class || type == boolean.class) { - return new BooleanChunkedSumOperator(name); - } else if (type == Byte.class || type == byte.class) { - return new ByteChunkedSumOperator(false, name); - } else if (type == Character.class || type == char.class) { - return new CharChunkedSumOperator(false, name); - } else if (type == Double.class || type == double.class) { - return new DoubleChunkedSumOperator(false, name); - } else if (type == Float.class || type == float.class) { - return new FloatChunkedSumOperator(false, name); - } else if (type == Integer.class || type == int.class) { - return new IntChunkedSumOperator(false, name); - } else if (type == Long.class || type == long.class) { - return new LongChunkedSumOperator(false, name); - } else if (type == Short.class || type == short.class) { - return new ShortChunkedSumOperator(false, name); - } else if (type == BigInteger.class) { - return new BigIntegerChunkedSumOperator(false, name); - } else if (type == BigDecimal.class) { - return new BigDecimalChunkedSumOperator(false, name); - } - throw new UnsupportedOperationException("Unsupported type " + type); - } - - static IterativeChunkedAggregationOperator getMinMaxChunked(Class type, String name, boolean minimum, - boolean isStreamOrAddOnly) { - if (!isStreamOrAddOnly) { - return new SsmChunkedMinMaxOperator(type, minimum, name); - } else { - if (type == Byte.class || type == byte.class) { - return new ByteChunkedAddOnlyMinMaxOperator(minimum, name); - } else if (type == Character.class || type == char.class) { - return new CharChunkedAddOnlyMinMaxOperator(minimum, name); - } else if (type == Double.class || type == double.class) { - return new DoubleChunkedAddOnlyMinMaxOperator(minimum, name); - } else if (type == Float.class || type == float.class) { - return new FloatChunkedAddOnlyMinMaxOperator(minimum, name); - } else if (type == Integer.class || type == int.class) { - return new IntChunkedAddOnlyMinMaxOperator(minimum, name); - } else if (type == Long.class || type == long.class || type == DateTime.class) { - return new LongChunkedAddOnlyMinMaxOperator(type, minimum, name); - } else if (type == Short.class || type == short.class) { - return new ShortChunkedAddOnlyMinMaxOperator(minimum, name); - } else if (type == Boolean.class || type == boolean.class) { - return new BooleanChunkedAddOnlyMinMaxOperator(minimum, name); - } else { - return new ObjectChunkedAddOnlyMinMaxOperator(type, minimum, name); - } - } - } - - static IterativeChunkedAggregationOperator getPercentileChunked(Class type, String name, double percentile, - boolean averageMedian) { - return new SsmChunkedPercentileOperator(type, percentile, averageMedian, name); - } - - static IterativeChunkedAggregationOperator getCountDistinctChunked(Class type, String name, boolean countNulls, - boolean exposeInternal, boolean isRollup) { - return DistinctOperatorFactory.createCountDistinct(type, name, countNulls, exposeInternal, isRollup); - } - - static IterativeChunkedAggregationOperator getDistinctChunked(Class type, String name, boolean countNulls, - boolean exposeInternal, boolean isRollup) { - return DistinctOperatorFactory.createDistinct(type, name, countNulls, exposeInternal, isRollup); - } - - static IterativeChunkedAggregationOperator getUniqueChunked(Class type, String name, boolean countNulls, - Object noKeyValue, Object nonUniqueValue, - boolean exposeInternal, boolean isRollup) { - return DistinctOperatorFactory.createUnique(type, name, countNulls, noKeyValue, nonUniqueValue, exposeInternal, - isRollup); - } - - static IterativeChunkedAggregationOperator getAbsSumChunked(Class type, String name) { - if (type == Boolean.class || type == boolean.class) { - return new BooleanChunkedSumOperator(name); - } else if (type == Byte.class || type == byte.class) { - return new ByteChunkedSumOperator(true, name); - } else if (type == Character.class || type == char.class) { - return new CharChunkedSumOperator(true, name); - } else if (type == Double.class || type == double.class) { - return new DoubleChunkedSumOperator(true, name); - } else if (type == Float.class || type == float.class) { - return new FloatChunkedSumOperator(true, name); - } else if (type == Integer.class || type == int.class) { - return new IntChunkedSumOperator(true, name); - } else if (type == Long.class || type == long.class) { - return new LongChunkedSumOperator(true, name); - } else if (type == Short.class || type == short.class) { - return new ShortChunkedSumOperator(true, name); - } else if (type == BigInteger.class) { - return new BigIntegerChunkedSumOperator(true, name); - } else if (type == BigDecimal.class) { - return new BigDecimalChunkedSumOperator(true, name); - } - throw new UnsupportedOperationException("Unsupported type " + type); - } - - static IterativeChunkedAggregationOperator getAvgChunked(Class type, String name, boolean exposeInternalColumns) { - if (type == Byte.class || type == byte.class) { - return new ByteChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == Character.class || type == char.class) { - return new CharChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == Double.class || type == double.class) { - return new DoubleChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == Float.class || type == float.class) { - return new FloatChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == Integer.class || type == int.class) { - return new IntChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == Long.class || type == long.class) { - return new LongChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == Short.class || type == short.class) { - return new ShortChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == BigInteger.class) { - return new BigIntegerChunkedAvgOperator(name, exposeInternalColumns); - } else if (type == BigDecimal.class) { - return new BigDecimalChunkedAvgOperator(name, exposeInternalColumns); - } else if (AvgState.class.isAssignableFrom(type)) { - throw new UnsupportedOperationException(); - } else if (AvgStateWithNan.class.isAssignableFrom(type)) { - throw new UnsupportedOperationException(); - } - throw new UnsupportedOperationException("Unsupported type " + type); - } - - static IterativeChunkedAggregationOperator getVarChunked(Class type, String name, boolean std, - boolean exposeInternalColumns) { - if (type == Byte.class || type == byte.class) { - return new ByteChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == Character.class || type == char.class) { - return new CharChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == Double.class || type == double.class) { - return new DoubleChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == Float.class || type == float.class) { - return new FloatChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == Integer.class || type == int.class) { - return new IntChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == Long.class || type == long.class) { - return new LongChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == Short.class || type == short.class) { - return new ShortChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == BigInteger.class) { - return new BigIntegerChunkedVarOperator(std, name, exposeInternalColumns); - } else if (type == BigDecimal.class) { - return new BigDecimalChunkedVarOperator(std, name, exposeInternalColumns); - } - throw new UnsupportedOperationException("Unsupported type " + type); - } - - @Override - boolean supportsRollup() { - return false; - } - - @Override - ReaggregatableStatefactory forRollup() { - throw new UnsupportedOperationException(); - } - - @Override - ReaggregatableStatefactory rollupFactory() { - throw new UnsupportedOperationException(); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.java deleted file mode 100644 index a74b5f8289e..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.java +++ /dev/null @@ -1,76 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import java.util.Objects; - -/** - * A Flavor of FirstBy that produces no values from the original table, only a named column of source keys. - */ -public class KeyOnlyFirstOrLastBySpec extends IterativeIndexSpec { - - private final String resultColumn; - private final AggType type; - - private static class MemoKey implements AggregationMemoKey { - private final String resultColumnName; - private final AggType type; - - private MemoKey(String resultColumnName, AggType type) { - this.resultColumnName = resultColumnName; - this.type = type; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - MemoKey memoKey = (MemoKey) o; - return Objects.equals(resultColumnName, memoKey.resultColumnName) && type == memoKey.type; - } - - @Override - public int hashCode() { - return Objects.hash(resultColumnName, type); - } - } - - public KeyOnlyFirstOrLastBySpec(String resultColumn, AggType type) { - super(false, false, 0); - this.resultColumn = resultColumn; - this.type = type; - - if (type != AggType.First && type != AggType.Last) { - throw new IllegalArgumentException( - "KeyOnlyFirstOrLastBySpec only support AggType.First and AggType.Last"); - } - } - - public String getResultColumn() { - return resultColumn; - } - - public boolean isLast() { - return type == AggType.Last; - } - - @Override - boolean supportsRollup() { - return false; - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(resultColumn, type); - } - - @Override - KeyOnlyFirstOrLastBySpec forRollup() { - throw new UnsupportedOperationException("KeyOnlyFirstBy does not support rollups."); - } - - @Override - public String toString() { - return "KeyOnlyFirstByStateFactory{resultColumn=" + resultColumn + '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java deleted file mode 100644 index 51c5e217419..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LastBySpecImpl.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public class LastBySpecImpl extends IterativeIndexSpec { - public LastBySpecImpl() { - this(false, false, 0); - } - - private LastBySpecImpl(boolean lowestRollup, boolean secondRollup, int rollupColumnIdentifier) { - super(lowestRollup, secondRollup, rollupColumnIdentifier); - } - - private static final AggregationMemoKey LAST_BY_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return LAST_BY_INSTANCE; - } - - - @Override - ReaggregatableStatefactory forRollup() { - return new LastBySpecImpl(true, false, 0); - } - - /** - * Sort the results by the original row key when aggregating on state. - */ - @Override - ReaggregatableStatefactory rollupFactory() { - return new SortedFirstOrLastByFactoryImpl(false, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); - } - - @Override - public String toString() { - if (!lowestRollup && !secondRollup) { - return "LastByStateFactory"; - } else { - return "LastByStateFactory{" + - "lowestRollup=" + lowestRollup + - ", secondRollup=" + secondRollup + - ", rollupColumnIdentifier=" + rollupColumnIdentifier + - '}'; - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.java deleted file mode 100644 index cff236bcaae..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * Minimum and Maximum aggregation factory. - * - * Operates in two modes, for non-refreshing tables it requires very little state (just the current minimum or maximum). - * - * For refreshing tables, it requires maintaining a TreeMap of values to counts; so that if the min/max value is removed - * we are able to identify the next lowest/highest value. - * - * You can use {@link AddOnlyMinMaxBySpecImpl} if you want to force add-only behavior. - * - */ -public class MinMaxBySpecImpl extends ReaggregatableStatefactory { - private final boolean minimum; - private final boolean requireAddOnly; - - /** - * Create a minBy or maxBy factory. - * - * @param minimum true if selecting the minimum value, false if selecting the maximum value. - */ - public MinMaxBySpecImpl(boolean minimum) { - this(minimum, false); - } - - /** - * Create a minBy or maxBy factory. - * - * @param minimum true if selecting the minimum value, false if selecting the maximum value. - * @param addOnly if true create a factory only suitable for add-only tables, if false the add-only factory will be - * created for non-refreshing tables and the general factory is created for refreshing tables - */ - MinMaxBySpecImpl(boolean minimum, boolean addOnly) { - this.minimum = minimum; - this.requireAddOnly = addOnly; - } - - public boolean isMinimum() { - return minimum; - } - - private static final AggregationMemoKey MIN_INSTANCE = new AggregationMemoKey() {}; - private static final AggregationMemoKey MAX_INSTANCE = new AggregationMemoKey() {}; - private static final AggregationMemoKey ADD_ONLY_MIN_INSTANCE = new AggregationMemoKey() {}; - private static final AggregationMemoKey ADD_ONLY_MAX_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - if (requireAddOnly) { - return minimum ? MIN_INSTANCE : MAX_INSTANCE; - } else { - return minimum ? ADD_ONLY_MIN_INSTANCE : ADD_ONLY_MAX_INSTANCE; - } - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return this; - } - - @Override - public String toString() { - return (minimum ? "Min" : "Max") + "ByStateFactoryImpl"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java deleted file mode 100644 index 67725feb195..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import static io.deephaven.engine.table.impl.by.IterativeOperatorSpec.getMinMaxChunked; - -/** - * Factory for iterative sum aggregations. - */ -public class MinMaxIterativeOperatorFactory implements IterativeChunkedOperatorFactory { - private final boolean minimum; - private final boolean isAddOnly; - - public MinMaxIterativeOperatorFactory(boolean minimum, boolean isAddOnly) { - this.minimum = minimum; - this.isAddOnly = isAddOnly; - } - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getMinMaxChunked(type, name, minimum, isAddOnly); - } - - @Override - public String toString() { - return minimum ? "Min" : "Max"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.java deleted file mode 100644 index 11dd67461c1..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.java +++ /dev/null @@ -1,75 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.table.Table; -import io.deephaven.time.DateTime; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import org.jetbrains.annotations.NotNull; - -import java.util.*; - -public class NonKeyColumnAggregationFactory implements AggregationContextFactory { - private final IterativeChunkedOperatorFactory iterativeChunkedOperatorFactory; - - public NonKeyColumnAggregationFactory(IterativeChunkedOperatorFactory iterativeChunkedOperatorFactory) { - this.iterativeChunkedOperatorFactory = iterativeChunkedOperatorFactory; - } - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - return getAllColumnOperators(table, groupByColumns, iterativeChunkedOperatorFactory); - } - - private static AggregationContext getAllColumnOperators(Table withView, String[] groupByNameArray, - IterativeChunkedOperatorFactory iterativeOperatorStateFactory) { - final Set groupByNames = new HashSet<>(Arrays.asList(groupByNameArray)); - final int operatorColumnCount = withView.getColumnSourceMap().size() - groupByNames.size(); - - final List operators = new ArrayList<>(operatorColumnCount); - final List> inputColumns = new ArrayList<>(operatorColumnCount); - final List inputNames = new ArrayList<>(operatorColumnCount); - - withView.getColumnSourceMap().forEach((name, columnSource) -> { - if (groupByNames.contains(name)) { - return; - } - - final Class type = columnSource.getType(); - - // For DateTime columns, the in-memory source uses longs internally, and all supported aggregations (i.e. - // min and max) work correctly against longs. - final ColumnSource inputSource = - columnSource.getType() == DateTime.class ? ReinterpretUtils.dateTimeToLongSource(columnSource) - : columnSource; - - final IterativeChunkedAggregationOperator chunkedOperator = - iterativeOperatorStateFactory.getChunkedOperator(type, name, false); - if (chunkedOperator != null) { - // noinspection unchecked - inputColumns.add(inputSource); - inputNames.add(name); - operators.add(chunkedOperator); - } - }); - - final String[][] inputNameArray = new String[inputNames.size()][1]; - for (int ii = 0; ii < inputNames.size(); ++ii) { - inputNameArray[ii][0] = inputNames.get(ii); - } - - // noinspection unchecked - return new AggregationContext( - operators.toArray( - IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY), - inputNameArray, - inputColumns.toArray(ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY)); - } - - @Override - public String toString() { - return iterativeChunkedOperatorFactory.toString(); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/NullAggregationSpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/NullAggregationSpecImpl.java deleted file mode 100644 index 02754438aad..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/NullAggregationSpecImpl.java +++ /dev/null @@ -1,10 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -class NullAggregationSpecImpl implements AggregationSpec { - private static final AggregationMemoKey NULL_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return NULL_INSTANCE; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.java deleted file mode 100644 index e4f9c0cb20e..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.base.verify.Require; - -import java.io.Serializable; -import java.util.Objects; - -public class PercentileBySpecImpl implements Serializable, AggregationSpec { - private final double percentile; - private final boolean averageMedian; - - public PercentileBySpecImpl(double percentile) { - this(percentile, false); - } - - public PercentileBySpecImpl(double percentile, boolean averageMedian) { - this.percentile = percentile; - this.averageMedian = averageMedian; - Require.inRange(percentile, 0.0, 1.0, "percentile"); - } - - private static class MemoKey implements AggregationMemoKey { - private final double percentile; - private final boolean averageMedian; - - private MemoKey(double percentile, boolean averageMedian) { - this.percentile = percentile; - this.averageMedian = averageMedian; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final MemoKey memoKey = (MemoKey) o; - return Double.compare(memoKey.percentile, percentile) == 0 && - averageMedian == memoKey.averageMedian; - } - - @Override - public int hashCode() { - return Objects.hash(percentile, averageMedian); - } - } - - public double getPercentile() { - return percentile; - } - - public boolean getAverageMedian() { - return averageMedian; - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(percentile, averageMedian); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java deleted file mode 100644 index efc0b1013e6..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * Factory for iterative sum aggregations. - */ -public class PercentileIterativeOperatorFactory implements IterativeChunkedOperatorFactory { - private final boolean averageMedian; - private final double percentile; - - public PercentileIterativeOperatorFactory(double percentile, boolean averageMedian) { - this.averageMedian = averageMedian; - this.percentile = percentile; - } - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return IterativeOperatorSpec.getPercentileChunked(type, name, percentile, averageMedian); - } - - @Override - public String toString() { - return averageMedian ? "Median" : "Percentile(" + percentile + ")"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PreviousStateProvider.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PreviousStateProvider.java deleted file mode 100644 index 11e0eadcff6..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/PreviousStateProvider.java +++ /dev/null @@ -1,5 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -interface PreviousStateProvider { - T prev(); -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.java deleted file mode 100644 index 508cb94ad7f..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.java +++ /dev/null @@ -1,27 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -public abstract class ReaggregatableStatefactory implements AggregationSpec { - /** - * Returns true if this state factory supports rollup. - * - * If this factory does not support rollup, calling forRollup and rollupFactory produce undefined results. - * - * @return true if forRollup() and rollupFactory() are implemented. - */ - abstract boolean supportsRollup(); - - /** - * Returns the lowest level state factory for rollup. - * - * This may differ from the regular factory in that often the result column is insufficient to perform a rollup (for - * example an average needs not just the result, but the count and sum). - */ - abstract ReaggregatableStatefactory forRollup(); - - /** - * Returns the factory used to reaggregate the lowest or intermediate levels into the next level. - * - * For example, a count factory should return a sum factory to roll up the counts by summation. - */ - abstract ReaggregatableStatefactory rollupFactory(); -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.java deleted file mode 100644 index 794922a5d24..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.java +++ /dev/null @@ -1,11 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -public interface ReaggregationIterativeOperator { - Class getStateType(); - - Class getFinalResultType(); - - T currentValue(STATE state); - - T prev(STATE state); -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RowStatus.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RowStatus.java deleted file mode 100644 index 93d71434066..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RowStatus.java +++ /dev/null @@ -1,9 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public enum RowStatus { - Unchanged, New, Modified, Removed -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.java deleted file mode 100644 index d95262d4268..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.java +++ /dev/null @@ -1,19 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public class SelectDistinctSpecImpl implements AggregationSpec { - private static final AggregationMemoKey SELECT_DISTINCT_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return SELECT_DISTINCT_INSTANCE; - } - - @Override - public String toString() { - return "SelectDistinctSpecImpl()"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstBy.java deleted file mode 100644 index 4b15d5fd784..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstBy.java +++ /dev/null @@ -1,11 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public final class SortedFirstBy extends SortedFirstOrLastByFactoryImpl { - public SortedFirstBy(String... sortColumnNames) { - super(true, sortColumnNames); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java deleted file mode 100644 index 91eb2e67665..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.java +++ /dev/null @@ -1,115 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.BaseTable; -import io.deephaven.engine.table.impl.StreamTableTools; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.chunk.ChunkType; -import io.deephaven.engine.table.impl.TupleSourceFactory; -import org.jetbrains.annotations.NotNull; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; - -public class SortedFirstOrLastByAggregationFactory implements AggregationContextFactory { - - private final boolean isFirst; - private final boolean isCombo; - private final String[] sortColumns; - - public SortedFirstOrLastByAggregationFactory(final boolean isFirst, final boolean isCombo, - final String... sortColumns) { - this.isFirst = isFirst; - this.isCombo = isCombo; - this.sortColumns = sortColumns; - } - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - final Set groupBySet = new HashSet<>(Arrays.asList(groupByColumns)); - return getAggregationContext(table, sortColumns, isFirst, isCombo, - table.getDefinition().getColumnNames().stream().filter(col -> !groupBySet.contains(col)) - .map(name -> new MatchPair(name, name)).toArray(MatchPair[]::new)); - } - - @NotNull - static AggregationContext getAggregationContext(@NotNull final Table table, - @NotNull final String[] sortColumns, - final boolean isFirst, - final boolean isCombo, - @NotNull final MatchPair[] resultNames) { - // noinspection unchecked - final ChunkSource.WithPrev[] inputSource = new ChunkSource.WithPrev[1]; - final IterativeChunkedAggregationOperator[] operator = new IterativeChunkedAggregationOperator[1]; - final String[][] name = new String[1][]; - - if (sortColumns.length == 1) { - final ColumnSource columnSource = table.getColumnSource(sortColumns[0]); - // noinspection unchecked - inputSource[0] = columnSource; - } else { - // create a tuple source, because our underlying SSA does not handle multiple sort columns - final ColumnSource[] sortColumnSources = new ColumnSource[sortColumns.length]; - for (int ii = 0; ii < sortColumnSources.length; ++ii) { - sortColumnSources[ii] = table.getColumnSource(sortColumns[ii]); - } - // noinspection unchecked - inputSource[0] = TupleSourceFactory.makeTupleSource(sortColumnSources); - } - - name[0] = sortColumns; - operator[0] = makeOperator(inputSource[0].getChunkType(), isFirst, isCombo, resultNames, table); - - return new AggregationContext(operator, name, inputSource); - } - - @Override - public String toString() { - return (isFirst ? "SortedFirstBy" : "SortedLastBy") + Arrays.toString(sortColumns); - } - - static IterativeChunkedAggregationOperator makeOperator(@NotNull final ChunkType chunkType, - final boolean isFirst, - final boolean isCombo, - @NotNull final MatchPair[] resultPairs, - @NotNull final Table sourceTable) { - final boolean isAddOnly = ((BaseTable) sourceTable).isAddOnly(); - final boolean isStream = StreamTableTools.isStream(sourceTable); - if (isAddOnly) { - // @formatter:off - switch (chunkType) { - case Boolean: throw new UnsupportedOperationException("Columns never use boolean chunks"); - case Char: return new CharAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); - case Byte: return new ByteAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); - case Short: return new ShortAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); - case Int: return new IntAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); - case Long: return new LongAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); - case Float: return new FloatAddOnlySortedFirstOrLastChunkedOperator( isFirst, resultPairs, sourceTable, null); - case Double: return new DoubleAddOnlySortedFirstOrLastChunkedOperator(isFirst, resultPairs, sourceTable, null); - case Object: return new ObjectAddOnlySortedFirstOrLastChunkedOperator(isFirst, resultPairs, sourceTable, null); - } - // @formatter:on - } - if (isStream) { - // @formatter:off - switch (chunkType) { - case Boolean: throw new UnsupportedOperationException("Columns never use boolean chunks"); - case Char: return new CharStreamSortedFirstOrLastChunkedOperator( isFirst, isCombo, resultPairs, sourceTable); - case Byte: return new ByteStreamSortedFirstOrLastChunkedOperator( isFirst, isCombo, resultPairs, sourceTable); - case Short: return new ShortStreamSortedFirstOrLastChunkedOperator( isFirst, isCombo, resultPairs, sourceTable); - case Int: return new IntStreamSortedFirstOrLastChunkedOperator( isFirst, isCombo, resultPairs, sourceTable); - case Long: return new LongStreamSortedFirstOrLastChunkedOperator( isFirst, isCombo, resultPairs, sourceTable); - case Float: return new FloatStreamSortedFirstOrLastChunkedOperator( isFirst, isCombo, resultPairs, sourceTable); - case Double: return new DoubleStreamSortedFirstOrLastChunkedOperator(isFirst, isCombo, resultPairs, sourceTable); - case Object: return new ObjectStreamSortedFirstOrLastChunkedOperator(isFirst, isCombo, resultPairs, sourceTable); - } - // @formatter:on - } - return new SortedFirstOrLastChunkedOperator(chunkType, isFirst, resultPairs, sourceTable); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.java deleted file mode 100644 index e60199e6ce8..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.base.verify.Require; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.Objects; - -public class SortedFirstOrLastByFactoryImpl extends IterativeIndexSpec implements Serializable { - - final private String[] sortColumnNames; - final private boolean minimum; - - SortedFirstOrLastByFactoryImpl(boolean minimum, String... sortColumnNames) { - this(minimum, false, false, 0, sortColumnNames); - } - - SortedFirstOrLastByFactoryImpl(boolean minimum, boolean firstRollup, boolean secondRollup, int rollupIdentifier, - String... sortColumnNames) { - super(firstRollup, secondRollup, rollupIdentifier); - Require.gtZero(sortColumnNames.length, "sortColumnNames.length"); - this.sortColumnNames = sortColumnNames; - this.minimum = minimum; - } - - public String[] getSortColumnNames() { - return sortColumnNames; - } - - private static final class MemoKey implements AggregationMemoKey { - private final boolean minimum; - private final String[] sortColumnNames; - - private MemoKey(boolean minimum, String[] sortColumnNames) { - this.minimum = minimum; - this.sortColumnNames = sortColumnNames; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final MemoKey memoKey = (MemoKey) o; - return minimum == memoKey.minimum && - Arrays.equals(sortColumnNames, memoKey.sortColumnNames); - } - - @Override - public int hashCode() { - int result = Objects.hash(minimum); - result = 31 * result + Arrays.hashCode(sortColumnNames); - return result; - } - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(minimum, sortColumnNames); - } - - @Override - ReaggregatableStatefactory forRollup() { - return new SortedFirstOrLastByFactoryImpl(minimum, true, false, 0, sortColumnNames); - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return new SortedFirstOrLastByFactoryImpl(minimum, false, true, 0, sortColumnNames); - } - - public boolean isSortedFirst() { - return minimum; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedLastBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedLastBy.java deleted file mode 100644 index c04abb26bba..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SortedLastBy.java +++ /dev/null @@ -1,11 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public final class SortedLastBy extends SortedFirstOrLastByFactoryImpl { - public SortedLastBy(String... sortColumnNames) { - super(false, sortColumnNames); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java deleted file mode 100644 index e9fde99526d..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdSpec.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * Factory for iterative standard deviation aggregations. - */ -public class StdSpec extends IterativeOperatorSpec { - public StdSpec() {} - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getVarChunked(type, name, true, exposeInternalColumns); - } - - - private static final AggregationMemoKey STD_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return STD_INSTANCE; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return this; - } - - @Override - public String toString() { - return "Std"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdState.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdState.java deleted file mode 100644 index 7f173ac9ca7..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdState.java +++ /dev/null @@ -1,63 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TableToolsShowControl; -import io.deephaven.engine.updategraph.LogicalClock; -import org.jetbrains.annotations.NotNull; - -@TableToolsShowControl(getWidth = 40) -class StdState implements PreviousStateProvider { - StdState(boolean previous) { - if (!previous) { - prevValue = createPrev(); - changeTime = -1; - } else { - prevValue = null; - changeTime = -1; - } - } - - @NotNull - StdState createPrev() { - return new StdState(true); - } - - // only used in the current state - private long changeTime; - private final StdState prevValue; - - protected double sum; - protected double sum2; - protected long nonNullCount; - - double currentValue() { - return Math.sqrt(sum2 / (nonNullCount - 1) - sum * sum / nonNullCount / (nonNullCount - 1)); - } - - @Override - public StdState prev() { - return prevValue; - } - - void checkUpdates() { - final long currentStep = LogicalClock.DEFAULT.currentStep(); - if (changeTime != currentStep) { - savePrevious(); - changeTime = currentStep; - } - } - - void savePrevious() { - prev().sum = sum; - prev().sum2 = sum2; - prev().nonNullCount = nonNullCount; - } - - @Override - public String toString() { - return "Std{" + - "sum=" + sum + - ", sum2=" + sum2 + - ", nonNull=" + nonNullCount + - '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdStateWithNan.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdStateWithNan.java deleted file mode 100644 index d1b4849c190..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StdStateWithNan.java +++ /dev/null @@ -1,37 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TableToolsShowControl; -import org.jetbrains.annotations.NotNull; - -@TableToolsShowControl(getWidth = 40) -class StdStateWithNan extends StdState { - protected long nanCount; - - StdStateWithNan(boolean previous) { - super(previous); - } - - @NotNull - @Override - StdStateWithNan createPrev() { - return new StdStateWithNan(true); - } - - double currentValue() { - if (nanCount > 0) { - return Double.NaN; - } - return Math.sqrt(sum2 / (nonNullCount - 1) - sum * sum / nonNullCount / (nonNullCount - 1)); - } - - @Override - void savePrevious() { - super.savePrevious(); - ((StdStateWithNan) prev()).nanCount = nanCount; - } - - @Override - public String toString() { - return "Std{" + "sum=" + sum + ", sum2=" + sum2 + ", nan=" + nanCount + ", nonNull=" + nonNullCount + '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SumSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SumSpec.java deleted file mode 100644 index b030a3fb995..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/SumSpec.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * Factory for iterative sum aggregations. - */ -public class SumSpec extends IterativeOperatorSpec { - public SumSpec() {} - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getSumChunked(type, name); - } - - private static final AggregationMemoKey SUM_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return SUM_INSTANCE; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return this; - } - - @Override - public String toString() { - return "Sum"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java deleted file mode 100644 index 8bede585dd1..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public class TrackingFirstBySpecImpl extends IterativeIndexSpec { - public TrackingFirstBySpecImpl() { - this(false, false, 0); - } - - private TrackingFirstBySpecImpl(boolean lowestRollup, boolean secondRollup, int rollupColumnIdentifier) { - super(lowestRollup, secondRollup, rollupColumnIdentifier); - } - - private static final AggregationMemoKey TRACKING_FIRSTBY_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return TRACKING_FIRSTBY_INSTANCE; - } - - - @Override - ReaggregatableStatefactory forRollup() { - return new TrackingFirstBySpecImpl(true, false, 0); - } - - /** - * Sort the results by the original row key when aggregating on state. - */ - @Override - ReaggregatableStatefactory rollupFactory() { - return new SortedFirstOrLastByFactoryImpl(true, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); - } - - @Override - public String toString() { - if (!lowestRollup && !secondRollup) { - return "TrackingFirstByStateFactory"; - } else { - return "TrackingFirstByStateFactory{" + - "lowestRollup=" + lowestRollup + - ", secondRollup=" + secondRollup + - ", rollupColumnIdentifier=" + rollupColumnIdentifier + - '}'; - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java deleted file mode 100644 index 6f003dc618c..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -public class TrackingLastBySpecImpl extends IterativeIndexSpec { - public TrackingLastBySpecImpl() { - this(false, false, 0); - } - - private TrackingLastBySpecImpl(boolean lowestRollup, boolean secondRollup, int rollupColumnIdentifier) { - super(lowestRollup, secondRollup, rollupColumnIdentifier); - } - - private static final AggregationMemoKey TRACKING_LASTBY_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return TRACKING_LASTBY_INSTANCE; - } - - - @Override - ReaggregatableStatefactory forRollup() { - return new TrackingLastBySpecImpl(true, false, 0); - } - - /** - * Sort the results by the original row key when aggregating on state. - */ - @Override - ReaggregatableStatefactory rollupFactory() { - return new SortedFirstOrLastByFactoryImpl(false, false, true, rollupColumnIdentifier, - ROW_REDIRECTION_PREFIX + rollupColumnIdentifier + RollupConstants.ROLLUP_COLUMN_SUFFIX); - } - - @Override - public String toString() { - if (!lowestRollup && !secondRollup) { - return "TrackingLastByStateFactory"; - } else { - return "TrackingLastByStateFactory{" + - "lowestRollup=" + lowestRollup + - ", secondRollup=" + secondRollup + - ", rollupColumnIdentifier=" + rollupColumnIdentifier + - '}'; - } - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java deleted file mode 100644 index 67cd7804094..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/UniqueSpec.java +++ /dev/null @@ -1,100 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import java.util.Objects; - -/** - * An Iterative state factory that displays the singular unique value of the items within a particular state, or default - * values if none are present, or the values are not unique. - */ -public class UniqueSpec extends IterativeOperatorSpec { - private final boolean secondRollup; - private final boolean countNulls; - private final Object noKeyValue; - private final Object nonUniqueValue; - - private static final class AggUniqueMemoKey implements AggregationMemoKey { - private final boolean countNulls; - private final Object noKeyValue; - private final Object nonUniqueValue; - - private AggUniqueMemoKey(boolean countNulls, Object noKeyValue, Object nonUniqueValue) { - this.countNulls = countNulls; - this.noKeyValue = noKeyValue; - this.nonUniqueValue = nonUniqueValue; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - AggUniqueMemoKey that = (AggUniqueMemoKey) o; - return countNulls == that.countNulls && Objects.equals(noKeyValue, that.noKeyValue) - && Objects.equals(nonUniqueValue, that.nonUniqueValue); - } - - @Override - public int hashCode() { - return Objects.hash(countNulls, noKeyValue, nonUniqueValue); - } - } - - UniqueSpec() { - this(false, false, null, null); - } - - UniqueSpec(boolean countNulls) { - this(false, countNulls, null, null); - } - - UniqueSpec(boolean countNulls, Object noKeyValue, Object nonUniqueValue) { - this(false, countNulls, noKeyValue, nonUniqueValue); - } - - private UniqueSpec(boolean secondRollup, boolean countNulls, Object noKeyValue, Object nonUniqueValue) { - this.secondRollup = secondRollup; - this.countNulls = countNulls; - this.noKeyValue = noKeyValue; - this.nonUniqueValue = nonUniqueValue; - } - - public Object getNoKeyValue() { - return noKeyValue; - } - - public Object getNonUniqueValue() { - return nonUniqueValue; - } - - @Override - public AggregationMemoKey getMemoKey() { - return new AggUniqueMemoKey(countNulls, noKeyValue, nonUniqueValue); - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - UniqueSpec forRollup() { - return this; - } - - @Override - UniqueSpec rollupFactory() { - return new UniqueSpec(true, countNulls, noKeyValue, nonUniqueValue); - } - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getUniqueChunked(type, name, countNulls, noKeyValue, nonUniqueValue, exposeInternalColumns, - secondRollup); - } - - public boolean countNulls() { - return countNulls; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java deleted file mode 100644 index 2d2eadd3106..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarSpec.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -/** - * Factory for iterative variance aggregations. - */ -public class VarSpec extends IterativeOperatorSpec { - public VarSpec() {} - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return getVarChunked(type, name, false, exposeInternalColumns); - } - - private static final AggregationMemoKey VAR_INSTANCE = new AggregationMemoKey() {}; - - @Override - public AggregationMemoKey getMemoKey() { - return VAR_INSTANCE; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return this; - } - - @Override - public String toString() { - return "Var"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarState.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarState.java deleted file mode 100644 index e4b8b5e522d..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarState.java +++ /dev/null @@ -1,63 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TableToolsShowControl; -import io.deephaven.engine.updategraph.LogicalClock; -import org.jetbrains.annotations.NotNull; - -@TableToolsShowControl(getWidth = 40) -class VarState implements PreviousStateProvider { - VarState(boolean previous) { - if (!previous) { - prevValue = createPrev(); - changeTime = -1; - } else { - prevValue = null; - changeTime = -1; - } - } - - @NotNull - VarState createPrev() { - return new VarState(true); - } - - // only used in the current state - private long changeTime; - private final VarState prevValue; - - protected double sum; - protected double sum2; - protected long nonNullCount; - - double currentValue() { - return sum2 / (nonNullCount - 1) - sum * sum / nonNullCount / (nonNullCount - 1); - } - - @Override - public VarState prev() { - return prevValue; - } - - void checkUpdates() { - final long currentStep = LogicalClock.DEFAULT.currentStep(); - if (changeTime != currentStep) { - savePrevious(); - changeTime = currentStep; - } - } - - void savePrevious() { - prev().sum = sum; - prev().sum2 = sum2; - prev().nonNullCount = nonNullCount; - } - - @Override - public String toString() { - return "Var{" + - "sum=" + sum + - ", sum2=" + sum2 + - ", nonNull=" + nonNullCount + - '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarStateWithNan.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarStateWithNan.java deleted file mode 100644 index 9582517bd4a..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/VarStateWithNan.java +++ /dev/null @@ -1,37 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.engine.util.TableToolsShowControl; -import org.jetbrains.annotations.NotNull; - -@TableToolsShowControl(getWidth = 40) -class VarStateWithNan extends VarState { - protected long nanCount; - - VarStateWithNan(boolean previous) { - super(previous); - } - - @NotNull - @Override - VarStateWithNan createPrev() { - return new VarStateWithNan(true); - } - - double currentValue() { - if (nanCount > 0) { - return Double.NaN; - } - return sum2 / (nonNullCount - 1) - sum * sum / nonNullCount / (nonNullCount - 1); - } - - @Override - void savePrevious() { - super.savePrevious(); - ((VarStateWithNan) prev()).nanCount = nanCount; - } - - @Override - public String toString() { - return "Var{" + "sum=" + sum + ", sum2=" + sum2 + ", nan=" + nanCount + ", nonNull=" + nonNullCount + '}'; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.java deleted file mode 100644 index 6fd78c7541a..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import java.util.Objects; - -public class WeightedAverageSpecImpl implements AggregationSpec { - private final String weightName; - - public WeightedAverageSpecImpl(String weightName) { - this.weightName = weightName; - } - - public String getWeightName() { - return weightName; - } - - private static class MemoKey implements AggregationMemoKey { - private final String weightColumn; - - private MemoKey(String weightColumn) { - this.weightColumn = weightColumn; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final MemoKey memoKey = (MemoKey) o; - return Objects.equals(weightColumn, memoKey.weightColumn); - } - - @Override - public int hashCode() { - return Objects.hash(weightColumn); - } - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(weightName); - } - - @Override - public String toString() { - return "WeightedAverageSpecImpl(" + weightName + ")"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.java deleted file mode 100644 index 60b91f93041..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import java.util.Objects; - -public class WeightedSumSpecImpl extends IterativeOperatorSpec { - private final String weightName; - - public WeightedSumSpecImpl(String weightName) { - super(); - this.weightName = weightName; - } - - public String getWeightName() { - return weightName; - } - - @Override - public IterativeChunkedAggregationOperator getChunkedOperator(Class type, String name, - boolean exposeInternalColumns) { - return null; - } - - @Override - boolean supportsRollup() { - return true; - } - - @Override - ReaggregatableStatefactory forRollup() { - return this; - } - - @Override - ReaggregatableStatefactory rollupFactory() { - return new SumSpec(); - } - - private static class MemoKey implements AggregationMemoKey { - private final String weightColumn; - - private MemoKey(String weightColumn) { - this.weightColumn = weightColumn; - } - - @Override - public boolean equals(Object o) { - if (this == o) - return true; - if (o == null || getClass() != o.getClass()) - return false; - final MemoKey memoKey = (MemoKey) o; - return Objects.equals(weightColumn, memoKey.weightColumn); - } - - @Override - public int hashCode() { - return Objects.hash(weightColumn); - } - } - - @Override - public AggregationMemoKey getMemoKey() { - return new MemoKey(weightName); - } -} From 1fc88edf6ccdc2048fa43d2db9be6fbfbd2b05a1 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 00:12:18 -0500 Subject: [PATCH 26/44] Replicate + spotless + doc regen --- .../doc/io/deephaven/api/agg/Aggregation.json | 11 ++++-- .../api/agg/Aggregation/Visitor.json | 2 +- .../api/agg/AggregationDescriptions.json | 10 +++++ .../api/agg/AggregationOptimizer.json | 4 +- .../deephaven/api/agg/AggregationOutputs.json | 7 +--- .../deephaven/api/agg/AggregationPairs.json | 11 ++++++ .../io/deephaven/api/agg/Aggregations.json | 11 ++++++ .../api/agg/Aggregations/Builder.json | 10 +++++ ...ggregation.json => ColumnAggregation.json} | 8 ++-- ...regations.json => ColumnAggregations.json} | 8 ++-- .../api/agg/ColumnAggregations/Builder.json | 11 ++++++ .../doc/io/deephaven/api/agg/FirstRowKey.json | 10 +++++ .../doc/io/deephaven/api/agg/LastRowKey.json | 10 +++++ .../api/agg/NormalAggregations/Builder.json | 11 ------ .../io/deephaven/api/agg/spec/AggSpec.json | 9 +++-- .../api/agg/spec/AggSpec/Visitor.json | 2 +- .../deephaven/api/agg/spec/AggSpecAbsSum.json | 1 + .../spec/AggSpecApproximatePercentile.json | 13 +++++++ .../io/deephaven/api/agg/spec/AggSpecAvg.json | 1 + .../deephaven/api/agg/spec/AggSpecBase.json | 2 +- .../api/agg/spec/AggSpecColumnReferences.json | 2 +- .../api/agg/spec/AggSpecCountDistinct.json | 1 + .../api/agg/spec/AggSpecDistinct.json | 1 + .../deephaven/api/agg/spec/AggSpecFirst.json | 1 + .../api/agg/spec/AggSpecFormula.json | 1 + .../deephaven/api/agg/spec/AggSpecGroup.json | 1 + .../deephaven/api/agg/spec/AggSpecLast.json | 1 + .../io/deephaven/api/agg/spec/AggSpecMax.json | 1 + .../deephaven/api/agg/spec/AggSpecMedian.json | 1 + .../io/deephaven/api/agg/spec/AggSpecMin.json | 1 + .../api/agg/spec/AggSpecPercentile.json | 1 + .../api/agg/spec/AggSpecSortedFirst.json | 1 + .../api/agg/spec/AggSpecSortedLast.json | 1 + .../io/deephaven/api/agg/spec/AggSpecStd.json | 1 + .../io/deephaven/api/agg/spec/AggSpecSum.json | 1 + .../api/agg/spec/AggSpecTDigest.json | 12 ++++++ .../deephaven/api/agg/spec/AggSpecUnique.json | 4 +- .../io/deephaven/api/agg/spec/AggSpecVar.json | 1 + .../deephaven/api/agg/spec/AggSpecWAvg.json | 1 + .../deephaven/api/agg/spec/AggSpecWSum.json | 1 + .../api/agg/util/PercentileOutput.json | 11 ++++++ .../AbstractColumnSource/IsSerializable.json | 7 ---- .../table/impl/HierarchicalTableInfo.json | 2 +- .../table/impl/MemoizedOperationKey.json | 4 +- .../engine/table/impl/QueryTable.json | 3 +- .../engine/table/impl/TupleSourceFactory.json | 2 +- .../TupleSourceCreator.json | 2 +- .../engine/table/impl/by/AbsSumSpec.json | 11 ------ .../impl/by/AddOnlyMinMaxBySpecImpl.json | 7 ---- .../table/impl/by/AggregationFactory.json | 38 ------------------- .../AggregationElement.json | 15 -------- .../AggregationElementImpl.json | 13 ------- .../CountAggregationElement.json | 13 ------- .../NullAggregationElement.json | 13 ------- .../table/impl/by/AggregationFormulaSpec.json | 10 ----- .../table/impl/by/AggregationGroupSpec.json | 8 ---- .../table/impl/by/AggregationMemoKey.json | 7 ---- .../table/impl/by/AggregationProcessor.json | 14 +++++++ .../engine/table/impl/by/AggregationSpec.json | 9 ----- .../table/impl/by/AggregationSpecAdapter.json | 10 ----- .../table/impl/by/ApproximatePercentile.json | 7 ++-- .../PercentileDefinition.json | 12 ------ .../engine/table/impl/by/AvgSpec.json | 11 ------ ...treamSortedFirstOrLastChunkedOperator.json | 2 +- ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../impl/by/CountAggregationFactory.json | 9 ----- .../engine/table/impl/by/CountBySpecImpl.json | 10 ----- .../table/impl/by/CountDistinctSpec.json | 11 ------ .../engine/table/impl/by/DistinctSpec.json | 11 ------ ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../engine/table/impl/by/FirstBySpecImpl.json | 10 ----- ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../table/impl/by/GroupByChunkedOperator.json | 2 +- ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../IterativeChunkedAggregationOperator.json | 2 +- .../table/impl/by/IterativeIndexSpec.json | 6 --- .../table/impl/by/IterativeOperatorSpec.json | 9 ----- .../impl/by/KeyOnlyFirstOrLastBySpec.json | 12 ------ .../engine/table/impl/by/LastBySpecImpl.json | 9 ----- ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../table/impl/by/MinMaxBySpecImpl.json | 11 ------ .../by/MinMaxIterativeOperatorFactory.json | 10 ----- .../by/NonKeyColumnAggregationFactory.json | 9 ----- ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../impl/by/PartitionByChunkedOperator.json | 2 +- .../table/impl/by/PercentileBySpecImpl.json | 10 ----- .../PercentileIterativeOperatorFactory.json | 10 ----- .../impl/by/ReaggregatableStatefactory.json | 6 --- .../by/ReaggregationIterativeOperator.json | 11 ------ .../engine/table/impl/by/RollupConstants.json | 7 ++++ .../engine/table/impl/by/RowStatus.json | 9 ----- .../table/impl/by/SelectDistinctSpecImpl.json | 9 ----- ...treamSortedFirstOrLastChunkedOperator.json | 2 +- .../engine/table/impl/by/SortedFirstBy.json | 6 --- ...SortedFirstOrLastByAggregationFactory.json | 9 ----- .../by/SortedFirstOrLastByFactoryImpl.json | 10 ----- .../engine/table/impl/by/SortedLastBy.json | 6 --- .../engine/table/impl/by/StdSpec.json | 11 ------ .../impl/by/StreamFirstChunkedOperator.json | 2 +- .../impl/by/StreamLastChunkedOperator.json | 2 +- .../engine/table/impl/by/SumSpec.json | 11 ------ .../impl/by/TDigestPercentileOperator.json | 9 +++-- .../impl/by/TrackingFirstBySpecImpl.json | 9 ----- .../table/impl/by/TrackingLastBySpecImpl.json | 9 ----- .../engine/table/impl/by/UniqueSpec.json | 13 ------- .../engine/table/impl/by/VarSpec.json | 11 ------ .../impl/by/WeightedAverageSpecImpl.json | 10 ----- .../table/impl/by/WeightedSumSpecImpl.json | 10 ----- .../table/impl/by/rollup/NullColumns.json | 13 +++++++ .../impl/by/rollup/NullColumns/Builder.json | 10 +++++ .../table/impl/by/rollup/Partition.json | 11 ++++++ .../impl/by/rollup/RollupAggregation.json | 11 ++++++ .../by/rollup/RollupAggregation/Visitor.json | 8 ++++ .../impl/by/rollup/RollupAggregationBase.json | 8 ++++ .../DistinctOperatorFactory.json | 8 ++-- .../ByteChunkedCountDistinctOperator.json | 2 +- .../ByteRollupCountDistinctOperator.json | 2 +- .../CharChunkedCountDistinctOperator.json | 2 +- .../CharRollupCountDistinctOperator.json | 2 +- .../DoubleChunkedCountDistinctOperator.json | 2 +- .../DoubleRollupCountDistinctOperator.json | 2 +- .../FloatChunkedCountDistinctOperator.json | 2 +- .../FloatRollupCountDistinctOperator.json | 2 +- .../IntChunkedCountDistinctOperator.json | 2 +- .../count/IntRollupCountDistinctOperator.json | 2 +- .../LongChunkedCountDistinctOperator.json | 2 +- .../LongRollupCountDistinctOperator.json | 2 +- .../ObjectChunkedCountDistinctOperator.json | 2 +- .../ObjectRollupCountDistinctOperator.json | 2 +- .../ShortChunkedCountDistinctOperator.json | 2 +- .../ShortRollupCountDistinctOperator.json | 2 +- .../distinct/ByteChunkedDistinctOperator.json | 2 +- .../distinct/ByteRollupDistinctOperator.json | 2 +- .../distinct/CharChunkedDistinctOperator.json | 2 +- .../distinct/CharRollupDistinctOperator.json | 2 +- .../DoubleChunkedDistinctOperator.json | 2 +- .../DoubleRollupDistinctOperator.json | 2 +- .../FloatChunkedDistinctOperator.json | 2 +- .../distinct/FloatRollupDistinctOperator.json | 2 +- .../distinct/IntChunkedDistinctOperator.json | 2 +- .../distinct/IntRollupDistinctOperator.json | 2 +- .../distinct/LongChunkedDistinctOperator.json | 2 +- .../distinct/LongRollupDistinctOperator.json | 2 +- .../ObjectChunkedDistinctOperator.json | 2 +- .../ObjectRollupDistinctOperator.json | 2 +- .../ShortChunkedDistinctOperator.json | 2 +- .../distinct/ShortRollupDistinctOperator.json | 2 +- .../unique/ByteChunkedUniqueOperator.json | 2 +- .../unique/ByteRollupUniqueOperator.json | 2 +- .../unique/CharChunkedUniqueOperator.json | 2 +- .../unique/CharRollupUniqueOperator.json | 2 +- .../unique/DoubleChunkedUniqueOperator.json | 2 +- .../unique/DoubleRollupUniqueOperator.json | 2 +- .../unique/FloatChunkedUniqueOperator.json | 2 +- .../unique/FloatRollupUniqueOperator.json | 2 +- .../unique/IntChunkedUniqueOperator.json | 2 +- .../unique/IntRollupUniqueOperator.json | 2 +- .../unique/LongChunkedUniqueOperator.json | 2 +- .../unique/LongRollupUniqueOperator.json | 2 +- .../unique/ObjectChunkedUniqueOperator.json | 2 +- .../unique/ObjectRollupUniqueOperator.json | 2 +- .../unique/ShortChunkedUniqueOperator.json | 2 +- .../unique/ShortRollupUniqueOperator.json | 2 +- .../impl/select/formula/FillContext.json | 6 +++ .../table/impl/select/formula/GetContext.json | 6 +++ .../table/impl/sources/FillFromUnordered.json | 8 ---- .../ImmutableBooleanArraySource.json | 9 ----- .../ImmutableDateTimeArraySource.json | 9 ----- .../tuplesource/TupleSourceCreatorImpl.json | 2 +- .../impl/util/freezeby/FreezeByOperator.json | 2 +- .../engine/util/TotalsTableBuilder.json | 18 ++++----- .../TotalsTableBuilder}/AggType.json | 10 ++--- .../doc/io/deephaven/plot/util/PlotUtils.json | 2 +- .../io/deephaven/annotations/TupleStyle.java | 6 +-- .../engine/table/impl/TupleSourceFactory.java | 2 +- .../table/impl/MemoizedOperationKey.java | 4 +- .../engine/table/impl/QueryTable.java | 6 +-- .../table/impl/by/AggregationContext.java | 6 +-- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- .../impl/by/FpChunkedNonNormalCounter.java | 3 +- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- .../IterativeChunkedAggregationOperator.java | 4 +- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- .../engine/table/impl/by/RollupConstants.java | 3 +- ...treamSortedFirstOrLastChunkedOperator.java | 2 +- .../DistinctOperatorFactory.java | 25 +++++++----- .../distinct/LongChunkedDistinctOperator.java | 3 +- .../distinct/LongRollupDistinctOperator.java | 3 +- .../unique/LongChunkedUniqueOperator.java | 3 +- .../unique/LongRollupUniqueOperator.java | 3 +- .../impl/select/AbstractRangeFilter.java | 3 +- .../engine/util/TotalsTableBuilder.java | 4 +- .../table/impl/QueryTableAggregationTest.java | 12 ++---- .../engine/table/impl/TestAggBy.java | 10 ++--- .../table/ops/ComboAggregateGrpcImpl.java | 10 ++--- 198 files changed, 416 insertions(+), 690 deletions(-) create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations/Builder.json rename Integrations/python/deephaven/doc/io/deephaven/api/agg/{NormalAggregation.json => ColumnAggregation.json} (52%) rename Integrations/python/deephaven/doc/io/deephaven/api/agg/{NormalAggregations.json => ColumnAggregations.json} (51%) create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations/Builder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations/Builder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/util/PercentileOutput.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/AbstractColumnSource/IsSerializable.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AbsSumSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElement.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElementImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/CountAggregationElement.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/NullAggregationElement.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationGroupSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationMemoKey.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationProcessor.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile/PercentileDefinition.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AvgSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountDistinctSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DistinctSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeIndexSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LastBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RollupConstants.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RowStatus.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstBy.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedLastBy.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StdSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SumSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/UniqueSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/VarSpec.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns/Builder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/Partition.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation/Visitor.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/FillContext.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/GetContext.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FillFromUnordered.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableBooleanArraySource.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDateTimeArraySource.json rename Integrations/python/deephaven/doc/io/deephaven/engine/{table/impl/by => util/TotalsTableBuilder}/AggType.json (52%) diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json index 9c34d71619c..4f69f568778 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json @@ -2,27 +2,32 @@ "className": "io.deephaven.api.agg.Aggregation", "methods": { "AggAbsSum": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", + "AggApproxPct": "*Overload 1* \n :param percentile: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param inputColumn: java.lang.String\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n :param inputColumn: java.lang.String\n :param compression: double\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation", "AggAvg": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggCount": ":param resultColumn: java.lang.String\n:return: io.deephaven.api.agg.Aggregation", "AggCountDistinct": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param countNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggDistinct": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggFirst": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", + "AggFirstRowKey": ":param resultColumn: java.lang.String\n:return: io.deephaven.api.agg.Aggregation", "AggFormula": ":param formula: java.lang.String\n:param formulaParam: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggGroup": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggLast": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", + "AggLastRowKey": ":param resultColumn: java.lang.String\n:return: io.deephaven.api.agg.Aggregation", "AggMax": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggMed": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param average: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggMin": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggPct": "*Overload 1* \n :param percentile: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param percentile: double\n :param average: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", + "AggPct": "*Overload 1* \n :param percentile: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param percentile: double\n :param average: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param inputColumn: java.lang.String\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n :param inputColumn: java.lang.String\n :param average: boolean\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation", "AggSortedFirst": "*Overload 1* \n :param sortedColumn: java.lang.String\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param sortedColumns: java.util.Collection\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggSortedLast": "*Overload 1* \n :param sortedColumn: java.lang.String\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param sortedColumns: java.util.Collection\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggStd": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggSum": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggUnique": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", + "AggTDigest": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param compression: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", + "AggUnique": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param nonUniqueSentinel: java.lang.Object\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggVar": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggWAvg": ":param weightColumn: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggWSum": ":param weightColumn: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "of": "*Overload 1* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pair: java.lang.String\n :return: io.deephaven.api.agg.NormalAggregation\n \n*Overload 2* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", + "PctOut": ":param percentile: double\n:param outputColumn: java.lang.String\n:return: io.deephaven.api.agg.util.PercentileOutput", + "of": "*Overload 1* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pair: java.lang.String\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param aggregations: io.deephaven.api.agg.Aggregation...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n Note: Java generics information - \n \n :param columnAggFactory: java.util.function.BiFunction\n :param inputColumn: java.lang.String\n :param inputs: INPUT_TYPE...\n :return: io.deephaven.api.agg.Aggregation", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, "path": "io.deephaven.api.agg.Aggregation", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json index a3dcf182e58..41f4259e649 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.api.agg.Aggregation$Visitor", "methods": { - "visit": "*Overload 1* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 2* \n :param normalAgg: io.deephaven.api.agg.NormalAggregation\n \n*Overload 3* \n :param normalAggs: io.deephaven.api.agg.NormalAggregations" + "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" }, "path": "io.deephaven.api.agg.Aggregation.Visitor", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json new file mode 100644 index 00000000000..8ba6fd4b69e --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.api.agg.AggregationDescriptions", + "methods": { + "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.Map\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.Map", + "visit": "*Overload 1* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 3* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 4* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 5* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + }, + "path": "io.deephaven.api.agg.AggregationDescriptions", + "text": "A visitor to describe the input and aggregation column name pairs for aggregations.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json index 1025d323f96..a16fe3dd4b6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json @@ -3,9 +3,9 @@ "methods": { "build": ":return: java.util.List", "of": "Optimizes a collection of aggregations by grouping like-specced aggregations together. The\n input order will be preserved based on the spec-encountered order.\n\n:param aggregations: (java.util.Collection) - the aggregations\n:return: (java.util.List) the optimized aggregations", - "visit": "*Overload 1* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 2* \n :param normalAgg: io.deephaven.api.agg.NormalAggregation\n \n*Overload 3* \n :param normalAggs: io.deephaven.api.agg.NormalAggregations" + "visit": "*Overload 1* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 3* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 4* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 5* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" }, "path": "io.deephaven.api.agg.AggregationOptimizer", - "text": "Optimizes a collection of aggregations by grouping like-speccedd aggregations together.", + "text": "Optimizes a collection of aggregations by grouping like-specced aggregations together.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json index 181e5d190da..e82296fcf91 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json @@ -1,10 +1,7 @@ { "className": "io.deephaven.api.agg.AggregationOutputs", - "methods": { - "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", - "visit": "*Overload 1* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 2* \n :param normalAgg: io.deephaven.api.agg.NormalAggregation\n \n*Overload 3* \n :param normalAggs: io.deephaven.api.agg.NormalAggregations" - }, + "methods": {}, "path": "io.deephaven.api.agg.AggregationOutputs", - "text": "A visitor to get the ordered output column names for aggregations.", + "text": "Helper to get the ordered output column names for aggregations.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json new file mode 100644 index 00000000000..64d83365a40 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.api.agg.AggregationPairs", + "methods": { + "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", + "outputsOf": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", + "visit": "*Overload 1* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 3* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 4* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 5* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + }, + "path": "io.deephaven.api.agg.AggregationPairs", + "text": "A visitor to get the ordered input/output column name pairs for aggregations.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json new file mode 100644 index 00000000000..0c60b0e4072 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.api.agg.Aggregations", + "methods": { + "aggregations": ":return: java.util.List", + "builder": ":return: io.deephaven.api.agg.Aggregations.Builder", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.api.agg.Aggregations", + "text": "Aggregations is an Aggregation that is a collection of two or more aggregations.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations/Builder.json new file mode 100644 index 00000000000..f4cab7fb50f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations/Builder.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.api.agg.Aggregations$Builder", + "methods": { + "addAggregations": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: io.deephaven.api.agg.Aggregations.Builder\n \n*Overload 2* \n :param aggregations: io.deephaven.api.agg.Aggregation...\n :return: io.deephaven.api.agg.Aggregations.Builder", + "addAllAggregations": ":param aggregations: java.lang.Iterable\n:return: io.deephaven.api.agg.Aggregations.Builder", + "build": ":return: io.deephaven.api.agg.Aggregations" + }, + "path": "io.deephaven.api.agg.Aggregations.Builder", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregation.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json similarity index 52% rename from Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregation.json rename to Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json index d9f486798e9..70b4c32f879 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregation.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json @@ -1,12 +1,12 @@ { - "className": "io.deephaven.api.agg.NormalAggregation", + "className": "io.deephaven.api.agg.ColumnAggregation", "methods": { - "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:param pair: io.deephaven.api.agg.Pair\n:return: io.deephaven.api.agg.NormalAggregation", + "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:param pair: io.deephaven.api.agg.Pair\n:return: io.deephaven.api.agg.ColumnAggregation", "pair": ":return: io.deephaven.api.agg.Pair", "spec": ":return: io.deephaven.api.agg.spec.AggSpec", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, - "path": "io.deephaven.api.agg.NormalAggregation", - "text": "A normal aggregation is an Aggregation that is composed of a spec and a pair.", + "path": "io.deephaven.api.agg.ColumnAggregation", + "text": "A ColumnAggregation is an Aggregation that is composed of a spec and a single input/output\n column pair.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json similarity index 51% rename from Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations.json rename to Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json index f708ae9a29e..a67af9562c9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json @@ -1,12 +1,12 @@ { - "className": "io.deephaven.api.agg.NormalAggregations", + "className": "io.deephaven.api.agg.ColumnAggregations", "methods": { - "builder": ":return: io.deephaven.api.agg.NormalAggregations.Builder", + "builder": ":return: io.deephaven.api.agg.ColumnAggregations.Builder", "pairs": ":return: java.util.List", "spec": ":return: io.deephaven.api.agg.spec.AggSpec", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, - "path": "io.deephaven.api.agg.NormalAggregations", - "text": "Normal aggregations is an Aggregation that is composed of a spec and multiple pairs.", + "path": "io.deephaven.api.agg.ColumnAggregations", + "text": "ColumnAggregations is an Aggregation that is composed of a spec and multiple input/output\n column pairs.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations/Builder.json new file mode 100644 index 00000000000..b4006539183 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations/Builder.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.api.agg.ColumnAggregations$Builder", + "methods": { + "addAllPairs": ":param elements: java.lang.Iterable\n:return: io.deephaven.api.agg.ColumnAggregations.Builder", + "addPairs": "*Overload 1* \n :param element: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.ColumnAggregations.Builder\n \n*Overload 2* \n :param elements: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.ColumnAggregations.Builder", + "build": ":return: io.deephaven.api.agg.ColumnAggregations", + "spec": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:return: io.deephaven.api.agg.ColumnAggregations.Builder" + }, + "path": "io.deephaven.api.agg.ColumnAggregations.Builder", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json new file mode 100644 index 00000000000..9121e5c11aa --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.api.agg.FirstRowKey", + "methods": { + "column": ":return: io.deephaven.api.ColumnName", + "of": "*Overload 1* \n :param name: io.deephaven.api.ColumnName\n :return: io.deephaven.api.agg.FirstRowKey\n \n*Overload 2* \n :param x: java.lang.String\n :return: io.deephaven.api.agg.FirstRowKey", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.api.agg.FirstRowKey", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json new file mode 100644 index 00000000000..8ab4d873e72 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.api.agg.LastRowKey", + "methods": { + "column": ":return: io.deephaven.api.ColumnName", + "of": "*Overload 1* \n :param name: io.deephaven.api.ColumnName\n :return: io.deephaven.api.agg.LastRowKey\n \n*Overload 2* \n :param x: java.lang.String\n :return: io.deephaven.api.agg.LastRowKey", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.api.agg.LastRowKey", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations/Builder.json deleted file mode 100644 index 07cba69fcc3..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/NormalAggregations/Builder.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.api.agg.NormalAggregations$Builder", - "methods": { - "addAllPairs": ":param elements: java.lang.Iterable\n:return: io.deephaven.api.agg.NormalAggregations.Builder", - "addPairs": "*Overload 1* \n :param element: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.NormalAggregations.Builder\n \n*Overload 2* \n :param elements: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.NormalAggregations.Builder", - "build": ":return: io.deephaven.api.agg.NormalAggregations", - "spec": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:return: io.deephaven.api.agg.NormalAggregations.Builder" - }, - "path": "io.deephaven.api.agg.NormalAggregations.Builder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json index ed0ea3c3e61..bbcb1f094e8 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json @@ -2,9 +2,11 @@ "className": "io.deephaven.api.agg.spec.AggSpec", "methods": { "absSum": ":return: io.deephaven.api.agg.spec.AggSpecAbsSum", - "aggregation": "*Overload 1* \n :param pair: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.NormalAggregation\n \n*Overload 2* \n :param pairs: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param pairs: java.util.Collection\n :return: io.deephaven.api.agg.Aggregation", + "aggregation": "*Overload 1* \n :param pair: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param pairs: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param pairs: java.util.Collection\n :return: io.deephaven.api.agg.Aggregation", + "approximatePercentile": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile", "avg": ":return: io.deephaven.api.agg.spec.AggSpecAvg", "countDistinct": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 2* \n :param countNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct", + "description": ":return: java.lang.String", "distinct": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecDistinct", "first": ":return: io.deephaven.api.agg.spec.AggSpecFirst", "formula": "*Overload 1* \n :param formula: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 2* \n :param formula: java.lang.String\n :param formulaParam: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula", @@ -18,13 +20,14 @@ "sortedLast": "*Overload 1* \n :param columns: java.lang.String...\n :return: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 2* \n :param columns: java.util.Collection\n :return: io.deephaven.api.agg.spec.AggSpecSortedLast", "std": ":return: io.deephaven.api.agg.spec.AggSpecStd", "sum": ":return: io.deephaven.api.agg.spec.AggSpecSum", - "unique": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecUnique", + "tDigest": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 2* \n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecTDigest", + "unique": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 2* \n :param includeNulls: boolean\n :param nonUniqueSentinel: java.lang.Object\n :return: io.deephaven.api.agg.spec.AggSpecUnique", "var": ":return: io.deephaven.api.agg.spec.AggSpecVar", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V", "wavg": ":param weightColumn: java.lang.String\n:return: io.deephaven.api.agg.spec.AggSpecWAvg", "wsum": ":param weightColumn: java.lang.String\n:return: io.deephaven.api.agg.spec.AggSpecWSum" }, "path": "io.deephaven.api.agg.spec.AggSpec", - "text": "An aggregation specification represents the configurable options for aggregations.", + "text": "An aggregation specification represents the configurable options for column aggregations.", "typeName": "interface" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json index 82212fa3606..8ac5667afe0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpec$Visitor", "methods": { - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 3* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 4* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 5* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 6* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 7* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 8* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 9* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 10* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 11* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 12* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 13* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 14* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 15* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 16* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 17* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 18* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 19* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 20* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" + "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 6* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 7* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 8* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 9* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 10* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 11* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 12* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 13* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 14* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 15* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 16* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 17* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 18* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 19* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 20* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 21* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 22* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" }, "path": "io.deephaven.api.agg.spec.AggSpec.Visitor", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json index 59fce8cc6cc..75457dbd478 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecAbsSum", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecAbsSum", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json new file mode 100644 index 00000000000..c7af90b967f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json @@ -0,0 +1,13 @@ +{ + "className": "io.deephaven.api.agg.spec.AggSpecApproximatePercentile", + "methods": { + "compression": "T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large.\n\n:return: (double) The T-Digest compression factor", + "description": ":return: java.lang.String", + "of": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile", + "percentile": "Percentile. Must be in range [0.0, 1.0].\n\n:return: (double) The percentile", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.api.agg.spec.AggSpecApproximatePercentile", + "text": "Approximate percentile aggregation using a T-Digest for calculation. Efficiently supports multiple output percentiles\n based on a single input column. May only be used on static or add-only tables.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json index 4177a4de9dd..07adebd765f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecAvg", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecAvg", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json index e11774b4bdc..ed7fd14cfc5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecBase", "methods": { - "aggregation": "*Overload 1* \n :param pair: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.NormalAggregation\n \n*Overload 2* \n :param pairs: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param pairs: java.util.Collection\n :return: io.deephaven.api.agg.Aggregation" + "aggregation": "*Overload 1* \n :param pair: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param pairs: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param pairs: java.util.Collection\n :return: io.deephaven.api.agg.Aggregation" }, "path": "io.deephaven.api.agg.spec.AggSpecBase", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json index 476e0278d06..3df76d3ae3b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json @@ -3,7 +3,7 @@ "methods": { "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:return: java.util.Set", "out": ":return: java.util.Set", - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 3* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 4* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 5* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 6* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 7* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 8* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 9* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 10* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 11* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 12* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 13* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 14* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 15* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 16* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 17* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 18* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 19* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 20* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" + "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 6* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 7* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 8* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 9* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 10* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 11* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 12* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 13* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 14* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 15* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 16* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 17* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 18* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 19* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 20* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 21* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 22* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" }, "path": "io.deephaven.api.agg.spec.AggSpecColumnReferences", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json index 581f901a302..7399690a121 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json @@ -2,6 +2,7 @@ "className": "io.deephaven.api.agg.spec.AggSpecCountDistinct", "methods": { "countNulls": ":return: boolean", + "description": ":return: java.lang.String", "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 2* \n :param countNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json index 54d2dd64b79..a78f5b47bd9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecDistinct", "methods": { + "description": ":return: java.lang.String", "includeNulls": ":return: boolean", "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecDistinct", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json index 71a8ad93b35..973aa755a0a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecFirst", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecFirst", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json index f89e78fc1f8..557840f37f3 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecFormula", "methods": { + "description": ":return: java.lang.String", "formula": ":return: java.lang.String", "formulaParam": ":return: java.lang.String", "of": "*Overload 1* \n :param formula: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 2* \n :param formula: java.lang.String\n :param formulaParam: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json index 6c63206e9a6..539e9131483 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecGroup", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecGroup", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json index c59a26992c7..7fd81792f01 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecLast", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecLast", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json index 26deebf4454..c08f4db1bbf 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecMax", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecMax", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json index 475f9d7cc91..99a88476eea 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json @@ -2,6 +2,7 @@ "className": "io.deephaven.api.agg.spec.AggSpecMedian", "methods": { "averageMedian": ":return: boolean", + "description": ":return: java.lang.String", "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 2* \n :param averageMedian: boolean\n :return: io.deephaven.api.agg.spec.AggSpecMedian", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json index 2d993d615fa..d76949f0c02 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecMin", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecMin", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json index a3ce1c54815..45eebefd3bc 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json @@ -2,6 +2,7 @@ "className": "io.deephaven.api.agg.spec.AggSpecPercentile", "methods": { "averageMedian": ":return: boolean", + "description": ":return: java.lang.String", "of": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 2* \n :param percentile: double\n :param averageMedian: boolean\n :return: io.deephaven.api.agg.spec.AggSpecPercentile", "percentile": ":return: double", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json index 7fb5e653132..fb53104186f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json @@ -3,6 +3,7 @@ "methods": { "builder": ":return: io.deephaven.api.agg.spec.AggSpecSortedFirst.Builder", "columns": ":return: java.util.List", + "description": ":return: java.lang.String", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, "path": "io.deephaven.api.agg.spec.AggSpecSortedFirst", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json index 31b014d47cd..93aa506d481 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json @@ -3,6 +3,7 @@ "methods": { "builder": ":return: io.deephaven.api.agg.spec.AggSpecSortedLast.Builder", "columns": ":return: java.util.List", + "description": ":return: java.lang.String", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, "path": "io.deephaven.api.agg.spec.AggSpecSortedLast", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json index 59792474a5d..4a080a83020 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecStd", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecStd", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json index a1cb4e4181f..aa2090c5c6c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecSum", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecSum", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json new file mode 100644 index 00000000000..15ad2c204bf --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json @@ -0,0 +1,12 @@ +{ + "className": "io.deephaven.api.agg.spec.AggSpecTDigest", + "methods": { + "compression": "T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large.\n\n:return: (double) The T-Digest compression factor", + "description": ":return: java.lang.String", + "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 2* \n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecTDigest", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.api.agg.spec.AggSpecTDigest", + "text": "Aggregates input column into a TDigest output column with the specified compression.\n \n May be used to implement parallel percentile calculations by splitting inputs and accumulating results into a single\n downstream TDigest.\n \n May only be used on static or add-only tables.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json index 4f027820885..89067f0e761 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json @@ -1,8 +1,10 @@ { "className": "io.deephaven.api.agg.spec.AggSpecUnique", "methods": { + "description": ":return: java.lang.String", "includeNulls": ":return: boolean", - "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecUnique", + "nonUniqueSentinel": ":return: java.lang.Object", + "of": "**Incompatible overloads text - text from the first overload:**\n\nSpecify a \"unique\" aggregation that does not treat null as a value for purposes of determining if the\n values in a group are unique. If a group is non-empty but contains only null values, its result will be\n null. If a group contains more than a single unique value, its result will also be null.\n\n*Overload 1* \n :return: (io.deephaven.api.agg.spec.AggSpecUnique) The \"unique\" aggregation specification\n \n*Overload 2* \n :param includeNulls: (boolean) - Whether null is treated as a value for determining if the values in a group are\n unique\n :param nonUniqueSentinel: (java.lang.Object) - Sentinel value to use if a group contains more than a single unique value\n :return: (io.deephaven.api.agg.spec.AggSpecUnique) The \"unique\" aggregation specification", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, "path": "io.deephaven.api.agg.spec.AggSpecUnique", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json index 5fcaf14c96e..4f71929e811 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecVar", "methods": { + "description": ":return: java.lang.String", "of": ":return: io.deephaven.api.agg.spec.AggSpecVar", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json index 4c99533a8c7..8722797fba1 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecWAvg", "methods": { + "description": ":return: java.lang.String", "of": ":param weight: io.deephaven.api.ColumnName\n:return: io.deephaven.api.agg.spec.AggSpecWAvg", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V", "weight": ":return: io.deephaven.api.ColumnName" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json index 29619fb67e5..0e8411a5684 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecWSum", "methods": { + "description": ":return: java.lang.String", "of": ":param weight: io.deephaven.api.ColumnName\n:return: io.deephaven.api.agg.spec.AggSpecWSum", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V", "weight": ":return: io.deephaven.api.ColumnName" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/util/PercentileOutput.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/util/PercentileOutput.json new file mode 100644 index 00000000000..0a6bc453aaa --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/util/PercentileOutput.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.api.agg.util.PercentileOutput", + "methods": { + "of": "*Overload 1* \n :param percentile: double\n :param output: io.deephaven.api.ColumnName\n :return: io.deephaven.api.agg.util.PercentileOutput\n \n*Overload 2* \n :param percentile: double\n :param output: java.lang.String\n :return: io.deephaven.api.agg.util.PercentileOutput", + "output": "Output column name.\n\n:return: (io.deephaven.api.ColumnName) The output column name", + "percentile": "Percentile. Must be in range [0.0, 1.0].\n\n:return: (double) The percentile" + }, + "path": "io.deephaven.api.agg.util.PercentileOutput", + "text": "Percentile and output column pair, used when specifying that a given input column should be aggregated into multiple\n exact or approximate percentiles.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/AbstractColumnSource/IsSerializable.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/AbstractColumnSource/IsSerializable.json deleted file mode 100644 index 828777ac547..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/AbstractColumnSource/IsSerializable.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.AbstractColumnSource$IsSerializable", - "methods": {}, - "path": "io.deephaven.engine.table.impl.AbstractColumnSource.IsSerializable", - "text": "We have a fair bit of internal state that must be serialized, but not all of our descendants in the class\n hierarchy should actually be sent over the wire. If you are implementing a class that should allow this to be\n serialized, then you must annotate it with an IsSerializable annotation, containing a value of true.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTableInfo.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTableInfo.json index b93fccf63ec..98cf534ed69 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTableInfo.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTableInfo.json @@ -7,6 +7,6 @@ "withColumnFormats": "Sets the column formats for the table info.\n\n:param columnFormats: (java.lang.String[]) - the column formats to set.\n:return: (io.deephaven.engine.table.impl.HierarchicalTableInfo) a copy of this HierarchicalTableInfo with column formats" }, "path": "io.deephaven.engine.table.impl.HierarchicalTableInfo", - "text": "A class that contains information required for a particular Hierarchical table type. (i.e\n tree tables or rollups)", + "text": "A class that contains information required for a particular Hierarchical table type, for example\n tree tables or rollups.", "typeName": "interface" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/MemoizedOperationKey.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/MemoizedOperationKey.json index 22863c9b967..b52339281e0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/MemoizedOperationKey.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/MemoizedOperationKey.json @@ -1,12 +1,12 @@ { "className": "io.deephaven.engine.table.impl.MemoizedOperationKey", "methods": { - "by": ":param aggregationSpec: io.deephaven.engine.table.impl.by.AggregationSpec\n:param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", + "aggBy": ":param aggregations: java.util.Collection\n:param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", "crossJoin": ":param rightTableCandidate: io.deephaven.engine.table.Table\n:param columnsToMatch: io.deephaven.engine.table.MatchPair[]\n:param columnsToAdd: io.deephaven.engine.table.MatchPair[]\n:param numRightBitsToReserve: int\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey.CrossJoin", "flatten": ":return: io.deephaven.engine.table.impl.MemoizedOperationKey", "partitionBy": ":param dropKeys: boolean\n:param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", "reverse": ":return: io.deephaven.engine.table.impl.MemoizedOperationKey", - "rollup": ":param aggregationFactory: io.deephaven.engine.table.impl.by.AggregationFactory\n:param columns: io.deephaven.engine.table.impl.select.SelectColumn[]\n:param includeConstituents: boolean\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", + "rollup": ":param aggregations: java.util.Collection\n:param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n:param includeConstituents: boolean\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", "symbolTable": ":param symbolTableSource: io.deephaven.engine.table.impl.sources.regioned.SymbolTableSource\n:param useLookupCaching: boolean\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", "treeTable": ":param idColumn: java.lang.String\n:param parentColumn: java.lang.String\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey", "wouldMatch": ":param pairs: io.deephaven.engine.table.WouldMatchPair...\n:return: io.deephaven.engine.table.impl.MemoizedOperationKey.WouldMatch" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json index 1df426c7601..b3ef7ae6ec9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json @@ -5,7 +5,6 @@ "aggBy": ":param aggregations: java.util.Collection\n:param groupByColumns: java.util.Collection\n:return: io.deephaven.engine.table.Table", "aj": "Looks up the columns in the rightTable that meet the match conditions in the columnsToMatch list. Matching is\n done exactly for the first n-1 columns and via a binary search for the last match pair. The columns of the\n original table are returned intact, together with the columns from rightTable defined in a comma separated list\n \"columnsToAdd\"\n\n:param rightTable: (io.deephaven.engine.table.Table) - The right side table on the join.\n:param columnsToMatch: (io.deephaven.engine.table.MatchPair[]) - A comma separated list of match conditions (\"leftColumn=rightColumn\" or\n \"columnFoundInBoth\")\n:param columnsToAdd: (io.deephaven.engine.table.MatchPair[]) - A comma separated list with the columns from the left side that need to be added to the right\n side as a result of the match.\n:param asOfMatchRule: io.deephaven.engine.table.Table.AsOfMatchRule\n:return: (io.deephaven.engine.table.Table) a new table joined according to the specification in columnsToMatch and columnsToAdd", "apply": "Applies a function to this table.\n \n This is useful if you have a reference to a table or a proxy and want to run a series of operations against the\n table without each individual operation resulting in an RMI.\n\nNote: Java generics information - \n\n:param function: (java.util.function.Function) - the function to run, its single argument will be this table\n:return: (R) the return value of function", - "by": ":param inputAggregationSpec: io.deephaven.engine.table.impl.by.AggregationSpec\n:param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn...\n:return: io.deephaven.engine.table.Table", "copy": "Copies this table, but with a new set of attributes.\n\n*Overload 1* \n :return: (io.deephaven.engine.table.Table) an identical table; but with a new set of attributes\n \n*Overload 2* \n :param copyAttributes: boolean\n :return: io.deephaven.engine.table.Table\n \n*Overload 3* \n :param definition: io.deephaven.engine.table.TableDefinition\n :param copyAttributes: boolean\n :return: io.deephaven.engine.table.Table", "countBy": ":param countColumnName: java.lang.String\n:param groupByColumns: io.deephaven.api.Selectable...\n:return: io.deephaven.engine.table.Table", "dateTimeColumnAsNanos": "Produce a new table with the same columns as this table, but with a new column presenting the specified DateTime\n column as a Long column (with each DateTime represented instead as the corresponding number of nanos since the\n epoch).\n \n NOTE: This is a really just an updateView(), and behaves accordingly for column ordering and (re)placement. This\n doesn't work on data that has been brought fully into memory (e.g. via select()). Use a view instead.\n\n:param dateTimeColumnName: (java.lang.String) - Name of date time column\n:param nanosColumnName: (java.lang.String) - Name of nanos column\n:return: (io.deephaven.engine.table.Table) The new table, constructed as explained above.", @@ -39,7 +38,7 @@ "releaseCachedResources": "Attempt to release cached resources held by this table. Unlike Table.close(), this must not render the table\n unusable for subsequent read operations. Implementations should be sure to call\n super.releaseCachedResources().", "renameColumns": ":param pairs: io.deephaven.engine.table.MatchPair...\n:return: io.deephaven.engine.table.Table", "reverse": "The reverse operation returns a new table that is the same as the original table, but the first row is last, and\n the last row is first. This is an internal API to be used by .raj(), but is accessible for unit tests.\n\n:return: (io.deephaven.engine.table.Table) the reversed table", - "rollup": "*Overload 1* \n :param aggregations: java.util.Collection\n :param includeConstituents: boolean\n :param groupByColumns: io.deephaven.api.Selectable...\n :return: io.deephaven.engine.table.Table\n \n*Overload 2* \n :param aggregationFactory: io.deephaven.engine.table.impl.by.AggregationFactory\n :param includeConstituents: boolean\n :param columns: io.deephaven.engine.table.impl.select.SelectColumn...\n :return: io.deephaven.engine.table.Table", + "rollup": ":param aggregations: java.util.Collection\n:param includeConstituents: boolean\n:param groupByColumns: io.deephaven.api.Selectable...\n:return: io.deephaven.engine.table.Table", "select": ":param columns: java.util.Collection\n:return: io.deephaven.engine.table.Table", "selectDistinct": ":param groupByColumns: java.util.Collection\n:return: io.deephaven.engine.table.Table", "setMemoizeResults": "For unit tests, provide a method to turn memoization on or off.\n\n:param memoizeResults: (boolean) - should results be memoized?\n:return: (boolean) the prior value", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory.json index a1eb2cf687b..2a143f13c2c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.engine.table.impl.TupleSourceFactory", "methods": { - "makeTupleSource": "Create a tuple source for the supplied array of column sources.\n\n:param columnSources: (io.deephaven.engine.table.ColumnSource...) - The column sources\n:return: (io.deephaven.engine.table.TupleSource) The tuple factory" + "makeTupleSource": "Create a tuple source for the supplied array of column sources.\n\nNote: Java generics information - \n\n:param columnSources: (io.deephaven.engine.table.ColumnSource...) - The column sources\n:return: (io.deephaven.engine.table.TupleSource) The tuple factory" }, "path": "io.deephaven.engine.table.impl.TupleSourceFactory", "text": "Factory for producing Deephaven engine TupleSource instances.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory/TupleSourceCreator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory/TupleSourceCreator.json index d5c17e5851f..2e9a1f76887 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory/TupleSourceCreator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/TupleSourceFactory/TupleSourceCreator.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.engine.table.impl.TupleSourceFactory$TupleSourceCreator", "methods": { - "makeTupleSource": "See TupleSourceFactory.makeTupleSource(ColumnSource[]).\n\n:param columnSources: io.deephaven.engine.table.ColumnSource...\n:return: io.deephaven.engine.table.TupleSource" + "makeTupleSource": "See TupleSourceFactory.makeTupleSource(ColumnSource[]).\n\nNote: Java generics information - \n\n:param columnSources: io.deephaven.engine.table.ColumnSource...\n:return: io.deephaven.engine.table.TupleSource" }, "path": "io.deephaven.engine.table.impl.TupleSourceFactory.TupleSourceCreator", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AbsSumSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AbsSumSpec.json deleted file mode 100644 index 1f20e2cdc7d..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AbsSumSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AbsSumSpec", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.AbsSumSpec", - "text": "Factory for iterative absolute sum aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.json deleted file mode 100644 index e00349288fc..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AddOnlyMinMaxBySpecImpl.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AddOnlyMinMaxBySpecImpl", - "methods": {}, - "path": "io.deephaven.engine.table.impl.by.AddOnlyMinMaxBySpecImpl", - "text": "If you've got a table that is grow only, this will do a min/max calculation without requiring any state. The\n limitation is that if you modify or remove a row it will throw an UnsupportedOperationException.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory.json deleted file mode 100644 index 3b168072f01..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationFactory", - "methods": { - "Agg": "Create an aggregation.\n\n*Overload 1* \n :param factory: (io.deephaven.engine.table.impl.by.AggregationSpec) - aggregation factory.\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 2* \n :param factory: (io.deephaven.engine.table.impl.by.AggregationSpec) - aggregation factory.\n :param matchPairs: (io.deephaven.engine.table.MatchPair...) - the columns to apply the aggregation to.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 3* \n :param factoryType: (io.deephaven.engine.table.impl.by.AggType) - aggregation factory type.\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 4* \n :param factoryType: (io.deephaven.engine.table.impl.by.AggType) - aggregation factory type.\n :param matchPairs: (io.deephaven.engine.table.MatchPair...) - the columns to apply the aggregation to.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggAbsSum": "Create an absolute sum aggregation, equivalent to Table.absSumBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggAvg": "Create an average aggregation, equivalent to Table.avgBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggCombo": "Create a new AggregationFactory suitable for passing to\n QueryTable.by(AggregationSpec, io.deephaven.engine.table.impl.select.SelectColumn...).\n\n:param aggregations: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement...) - the aggregations to compute\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory) a new table with the specified aggregations.", - "AggCount": "Create an count aggregation, equivalent to Table.countBy(String).\n\n:param resultColumn: (java.lang.String) - the name of the result column containing the count of each group\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggCountDistinct": "Create a distinct count aggregation.\n\n The output column contains the number of distinct values for the input column in that group.\n\n*Overload 1* \n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...). Null values\n are not counted.\n \n*Overload 2* \n :param countNulls: (boolean) - if true null values are counted as a distinct value, otherwise null values are ignored\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggDistinct": "Create a distinct aggregation.\n\n The output column contains a Vector with the distinct values for the input column within the group.\n\n*Overload 1* \n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...). Null values\n are ignored.\n \n*Overload 2* \n :param countNulls: (boolean) - if true, then null values are included in the result, otherwise null values are ignored\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggFirst": "Create a first aggregation, equivalent to Table.firstBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggFormula": "Create a formula aggregation.\n\n:param formula: (java.lang.String) - the formula to apply to each group\n:param formulaParam: (java.lang.String) - the parameter name within the formula\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggGroup": "Create a group aggregation, equivalent to Table.groupBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggLast": "Create a last aggregation, equivalent to Table.lastBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggMax": "Create a maximum aggregation, equivalent to Table.maxBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggMed": "Create a median aggregation, equivalent to Table.medianBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggMin": "Create a minimum aggregation, equivalent to Table.minBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggPct": "Create a percentile aggregation.\n\n*Overload 1* \n :param percentile: (double) - the percentile to calculate\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 2* \n :param percentile: (double) - the percentile to calculate\n :param averageMedian: (boolean) - if true, then when the upper values and lower values have an equal size; average the highest\n lower value and lowest upper value to produce the median value for integers, longs, doubles, and floats\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggSortedFirst": "Create a sorted first aggregation, equivalent to SortedBy.sortedFirstBy(io.deephaven.engine.table.Table, java.lang.String).\n\n*Overload 1* \n :param sortColumn: (java.lang.String) - the column to sort by\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 2* \n :param sortColumns: (java.lang.String[]) - the column to sort by\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggSortedLast": "Create a sorted last aggregation, equivalent to SortedBy.sortedLastBy(io.deephaven.engine.table.Table, java.lang.String).\n\n*Overload 1* \n :param sortColumn: (java.lang.String) - the column to sort by\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 2* \n :param sortColumns: (java.lang.String[]) - the columns to sort by\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggStd": "Create a standard deviation aggregation, equivalent to Table.stdBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggSum": "Create a summation aggregation, equivalent to Table.sumBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggUnique": "Create a Unique aggregation.\n\n The output column contains a value of the same type as the input column which contains\n\n* The \"no key value\" - if there are no values present\n* The single unique value - if there is only a single value present\n* The \"non unique value\" - if there are more than 1 distinct values present\n\n*Overload 1* \n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)\n \n*Overload 2* \n :param countNulls: (boolean) - if true, then null values are included in the result, otherwise null values are ignored\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...). Output\n columns contain null if there are no values present or there are more than 1 distinct values present.\n \n*Overload 3* \n :param countNulls: (boolean) - if true, then null values are included in the result, otherwise null values are ignored\n :param noKeyValue: (java.lang.Object) - the value to use if there are no values present\n :param nonUniqueValue: (java.lang.Object) - the value to use if there are more than 1 values present\n :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggVar": "Create a variance aggregation, equivalent to Table.varBy(String...).\n\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggWAvg": "Create a weighted average aggregation, equivalent to Table.wavgBy(String, String...).\n\n:param weight: (java.lang.String) - the name of the column to use as the weight for the average\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "AggWSum": "Create a weighted sum aggregation, equivalent to Table.wsumBy(String, String...).\n\n:param weight: (java.lang.String) - the name of the column to use as the weight for the sum\n:param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have\n the same name, then the column name can be specified.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...)", - "forRollup": ":param includeConstituents: boolean\n:return: io.deephaven.engine.table.impl.by.AggregationFactory", - "getMatchPairs": ":return: java.util.List", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "makeAggregationContextFactory": ":return: io.deephaven.engine.table.impl.by.AggregationContextFactory", - "rollupFactory": "Create a factory for performing rollups.\n\n:return: io.deephaven.engine.table.impl.by.AggregationFactory", - "toString": ":return: java.lang.String", - "withNulls": "Create a new factory that will have columns with all null values.\n\n Used by rollup to empty out unused grouping columns.\n\n:param nullColumns: (java.util.Map>) - a map of column names to types.\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory) a new AggregationFactory that will produce null values for the given columns." - }, - "path": "io.deephaven.engine.table.impl.by.AggregationFactory", - "text": "The AggregationFactory combines one or more aggregations into an AggregationSpec for use internally by the\n implementation of Table.aggBy(io.deephaven.api.agg.Aggregation).\n\n \n The intended use of this class is to call the AggCombo(AggregationElement...) method with a set of\n aggregations defined by:\n \n* AggMin(java.lang.String...)\n* AggMax(java.lang.String...)\n* AggSum(java.lang.String...)\n* AggAbsSum(java.lang.String...)\n* AggVar(java.lang.String...)\n* AggAvg(java.lang.String...)\n* AggWAvg(java.lang.String, java.lang.String...)\n* AggWSum(java.lang.String, java.lang.String...)\n* AggMed(java.lang.String...)\n* AggPct(double, java.lang.String...)\n* AggStd(java.lang.String...)\n* AggFirst(java.lang.String...)\n* AggLast(java.lang.String...)\n* AggCount(java.lang.String)\n* AggCountDistinct(java.lang.String...)\n* AggDistinct(java.lang.String...)\n* AggGroup(java.lang.String...)\n* AggSortedFirst(java.lang.String, java.lang.String...)\n* AggSortedLast(java.lang.String, java.lang.String...)", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElement.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElement.json deleted file mode 100644 index 4edd506c07f..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElement.json +++ /dev/null @@ -1,15 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationFactory$AggregationElement", - "methods": { - "convert": "Converts the aggregations leaving singular aggregations as they are.\n\n \n Note: The results will preserve the intended order of the inputs.\n\n:param aggregations: (java.util.Collection) - The aggregation\n:return: (java.util.List) A list of aggregation elements", - "forRollup": ":return: io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement", - "getMemoKey": ":return: io.deephaven.engine.table.impl.by.AggregationMemoKey", - "getResultPairs": ":return: io.deephaven.engine.table.MatchPair[]", - "getSourceColumns": ":return: java.lang.String[]", - "getSpec": ":return: io.deephaven.engine.table.impl.by.AggregationSpec", - "of": "Converts an Aggregation to an AggregationFactory.AggregationElement.\n\n:param aggregation: (io.deephaven.api.agg.Aggregation) - The aggregation\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) The aggregation element", - "optimizeAndConvert": "Converts and optimizes the aggregations, collapsing relevant aggregations into single\n elements where applicable.\n\n \n Note: due to the optimization, the aggregation elements may not be in the same order as specified in\n aggregations.\n\n:param aggregations: (java.util.Collection) - The aggregation\n:return: (java.util.List) A list of aggregation elements" - }, - "path": "io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElementImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElementImpl.json deleted file mode 100644 index 0990ac73e14..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/AggregationElementImpl.json +++ /dev/null @@ -1,13 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationFactory$AggregationElementImpl", - "methods": { - "forRollup": ":return: io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement", - "getMemoKey": ":return: io.deephaven.engine.table.impl.by.AggregationMemoKey", - "getResultPairs": ":return: io.deephaven.engine.table.MatchPair[]", - "getSourceColumns": ":return: java.lang.String[]", - "getSpec": ":return: io.deephaven.engine.table.impl.by.AggregationSpec", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElementImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/CountAggregationElement.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/CountAggregationElement.json deleted file mode 100644 index 3c748c7ca72..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/CountAggregationElement.json +++ /dev/null @@ -1,13 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationFactory$CountAggregationElement", - "methods": { - "forRollup": ":return: io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement", - "getMemoKey": ":return: io.deephaven.engine.table.impl.by.AggregationMemoKey", - "getResultPairs": ":return: io.deephaven.engine.table.MatchPair[]", - "getSourceColumns": ":return: java.lang.String[]", - "getSpec": ":return: io.deephaven.engine.table.impl.by.AggregationSpec", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.AggregationFactory.CountAggregationElement", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/NullAggregationElement.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/NullAggregationElement.json deleted file mode 100644 index 2d1753f10c6..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFactory/NullAggregationElement.json +++ /dev/null @@ -1,13 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationFactory$NullAggregationElement", - "methods": { - "forRollup": ":return: io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement", - "getMemoKey": ":return: io.deephaven.engine.table.impl.by.AggregationMemoKey", - "getResultPairs": ":return: io.deephaven.engine.table.MatchPair[]", - "getSourceColumns": ":return: java.lang.String[]", - "getSpec": ":return: io.deephaven.engine.table.impl.by.AggregationSpec", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.AggregationFactory.NullAggregationElement", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.json deleted file mode 100644 index 61dd3581df5..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationFormulaSpec.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationFormulaSpec", - "methods": { - "getColumnParamName": ":return: java.lang.String", - "getFormula": ":return: java.lang.String", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized." - }, - "path": "io.deephaven.engine.table.impl.by.AggregationFormulaSpec", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationGroupSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationGroupSpec.json deleted file mode 100644 index 7349161fe74..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationGroupSpec.json +++ /dev/null @@ -1,8 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationGroupSpec", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized." - }, - "path": "io.deephaven.engine.table.impl.by.AggregationGroupSpec", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationMemoKey.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationMemoKey.json deleted file mode 100644 index 2ce4303346d..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationMemoKey.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationMemoKey", - "methods": {}, - "path": "io.deephaven.engine.table.impl.by.AggregationMemoKey", - "text": "Marker interface for AggregationStates to use for memoization keys.", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationProcessor.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationProcessor.json new file mode 100644 index 00000000000..27aca9c19a2 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationProcessor.json @@ -0,0 +1,14 @@ +{ + "className": "io.deephaven.engine.table.impl.by.AggregationProcessor", + "methods": { + "forAggregation": "Convert a collection of aggregations to an AggregationContextFactory.\n\n:param aggregations: (java.util.Collection) - The aggregations. Must not be further mutated by the caller. Will not be\n mutated by AggregationProcessor.\n:return: (io.deephaven.engine.table.impl.by.AggregationContextFactory) The AggregationContextFactory", + "forRollupBase": "Convert a collection of aggregations to an AggregationContextFactory for use in\n computing the base level of a rollup.\n\n:param aggregations: (java.util.Collection) - The aggregations. Must not be further mutated by the caller. Will not be\n mutated by AggregationProcessor.\n:param includeConstituents: (boolean) - Whether constituents should be included via a partition aggregation\n:return: (io.deephaven.engine.table.impl.by.AggregationContextFactory) The AggregationContextFactory", + "forRollupReaggregated": "Convert a collection of aggregations to an AggregationContextFactory for use in\n computing a reaggregated level of a rollup.\n\n:param aggregations: (java.util.Collection) - The aggregations. Must not be further mutated by the caller. Will not be\n mutated by AggregationProcessor.\n:param nullColumns: (java.util.Map>) - Map of group-by column names and data types to aggregate with a null-column aggregation\n:return: (io.deephaven.engine.table.impl.by.AggregationContextFactory) The AggregationContextFactory", + "forSelectDistinct": "Create a trivial AggregationContextFactory to implement select distinct.\n\n:return: (io.deephaven.engine.table.impl.by.AggregationContextFactory) The AggregationContextFactory", + "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumnNames: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", + "toString": ":return: java.lang.String" + }, + "path": "io.deephaven.engine.table.impl.by.AggregationProcessor", + "text": "Conversion tool to generate an AggregationContextFactory for a collection of aggregations.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpec.json deleted file mode 100644 index fd9175cc73c..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpec.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationSpec", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized." - }, - "path": "io.deephaven.engine.table.impl.by.AggregationSpec", - "text": "AggregationSpecs are passed to Table.groupBy(java.util.Collection) operation, and used to supply operation-specific instructions.", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.json deleted file mode 100644 index fdd418b0941..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationSpecAdapter.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationSpecAdapter", - "methods": { - "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:return: io.deephaven.engine.table.impl.by.AggregationSpec", - "out": ":return: io.deephaven.engine.table.impl.by.AggregationSpec", - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 3* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 4* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 5* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 6* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 7* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 8* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 9* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 10* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 11* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 12* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 13* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 14* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 15* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 16* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 17* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 18* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 19* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 20* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" - }, - "path": "io.deephaven.engine.table.impl.by.AggregationSpecAdapter", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile.json index d21985d1e2b..39268bdaa26 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile.json @@ -1,11 +1,10 @@ { "className": "io.deephaven.engine.table.impl.by.ApproximatePercentile", "methods": { - "accumulateDigests": "Accumulate an Vector of TDigests into a single new TDigest.\n\n \n Accumulate the digests within the Vector into a single TDigest. The compression factor is one third of the\n compression factor of the first digest within the array. If the array has only a single element, then that\n element is returned. If a null array is passed in, null is returned.\n \n\n This function is intended to be used for parallelization. The first step is to independently compute approximate\n percentiles with an exposed digest column using your desired buckets. Next, call Table.groupBy(String...)\n to produce arrays of Digests for each relevant bucket. Once the arrays are created, use this function to\n accumulate the arrays of digests within an Table.update(String...) statement. Finally, you may call the\n TDigest quantile function (or others) to produce the desired approximate percentile.\n \n\n:param array: (io.deephaven.vector.ObjectVector) - an array of TDigests\n:return: (com.tdunning.math.stats.TDigest) the accumulated TDigests", - "approximatePercentile": "Compute the approximate percentiles for the table.\n\n*Overload 1* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param percentile: (double) - the percentile to compute for each column\n :return: (io.deephaven.engine.table.Table) a single row table with double columns representing the approximate percentile for each column of the\n input table\n \n*Overload 2* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param percentile: (double) - the percentile to compute for each column\n :param groupByColumns: (java.lang.String...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a with the groupByColumns and double columns representing the approximate percentile for each remaining\n column of the input table\n \n*Overload 3* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param percentile: (double) - the percentile to compute for each column\n :param groupByColumns: (io.deephaven.engine.table.impl.select.SelectColumn...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a with the groupByColumns and double columns representing the approximate percentile for each remaining\n column of the input table\n \n*Overload 4* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param compression: (double) - the t-digest compression parameter\n :param percentile: (double) - the percentile to compute for each column\n :param groupByColumns: (io.deephaven.engine.table.impl.select.SelectColumn...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a with the groupByColumns and double columns representing the approximate percentile for each remaining\n column of the input table", - "approximatePercentiles": "Compute a set of approximate percentiles for input according to the definitions in percentileDefinitions.\n\n*Overload 1* \n :param input: (io.deephaven.engine.table.Table) - the table to compute approximate percentiles for\n :param percentileDefinitions: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) - the compression factor, and map of input columns to output columns\n :param groupByColumns: (io.deephaven.engine.table.impl.select.SelectColumn...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a table containing the groupByColumns and the approximate percentiles\n \n*Overload 2* \n :param input: (io.deephaven.engine.table.Table) - the table to compute approximate percentiles for\n :param percentileDefinitions: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) - the compression factor, and map of input columns to output columns\n :param groupByColumns: (java.lang.String...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a table containing the groupByColumns and the approximate percentiles\n \n*Overload 3* \n :param input: (io.deephaven.engine.table.Table) - the table to compute approximate percentiles for\n :param percentileDefinitions: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) - the compression factor, and map of input columns to output columns\n :return: (io.deephaven.engine.table.Table) a table containing a single row with the the approximate percentiles" + "accumulateDigests": "Accumulate a Vector of TDigests into a single new TDigest.\n\n \n Accumulate the digests within the Vector into a single TDigest. The compression factor is one third of the\n compression factor of the first digest within the array. If the array has only a single element, then that\n element is returned. If a null array is passed in, null is returned.\n \n\n This function is intended to be used for parallelization. The first step is to independently expose a T-Digest\n aggregation column with the appropriate compression factor on each of a set of sub-tables, using\n Aggregation.AggTDigest(java.lang.String...) and Table.aggBy(io.deephaven.api.agg.Aggregation). Next, call Table.groupBy(String...) to produce\n arrays of Digests for each relevant bucket. Once the arrays are created, use this function to accumulate the\n arrays of digests within an Table.update(java.util.Collection) statement. Finally, you may call the TDigest quantile function\n (or others) to produce the desired approximate percentile.\n \n\n:param array: (io.deephaven.vector.ObjectVector) - an array of TDigests\n:return: (com.tdunning.math.stats.TDigest) the accumulated TDigests", + "approximatePercentileBy": "Compute the approximate percentiles for the table.\n\n*Overload 1* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param percentile: (double) - the percentile to compute for each column\n :return: (io.deephaven.engine.table.Table) a single row table with double columns representing the approximate percentile for each column of the\n input table\n \n*Overload 2* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param percentile: (double) - the percentile to compute for each column\n :param groupByColumns: (java.lang.String...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a with the groupByColumns and double columns representing the approximate percentile for each remaining\n column of the input table\n \n*Overload 3* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param percentile: (double) - the percentile to compute for each column\n :param groupByColumns: (io.deephaven.engine.table.impl.select.SelectColumn...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a with the groupByColumns and double columns representing the approximate percentile for each remaining\n column of the input table\n \n*Overload 4* \n :param input: (io.deephaven.engine.table.Table) - the input table\n :param compression: (double) - the t-digest compression parameter\n :param percentile: (double) - the percentile to compute for each column\n :param groupByColumns: (io.deephaven.engine.table.impl.select.SelectColumn...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) a with the groupByColumns and double columns representing the approximate percentile for each remaining\n column of the input table" }, "path": "io.deephaven.engine.table.impl.by.ApproximatePercentile", - "text": "Generate approximate percentile aggregations of a table.\n\n \n The underlying data structure and algorithm used is a t-digest as described at https://github.com/tdunning/t-digest,\n which has a \"compression\" parameter that determines the size of the retained values. From the t-digest documentation,\n \"100 is a common value for normal uses. 1000 is extremely large. The number of centroids retained will be a\n smallish (usually less than 10) multiple of this number."e;\n \n\n All input columns are cast to doubles and the result columns are doubles.\n \n\n The input table must be add only, if modifications or removals take place; then an UnsupportedOperationException is\n thrown. For tables with adds and removals you must use exact percentiles with\n AggregationFactory.AggPct(double, java.lang.String...).\n \n\n You may compute either one approximate percentile or several approximate percentiles at once. For example, to compute\n the 95th percentile of all other columns, by the \"Sym\" column you would call:\n \n \n ApproximatePercentile.approximatePercentile(input, 0.95, \"Sym\")\n \n\n If you need to compute several percentiles, it is more efficient to compute them simultaneously. For example, this\n example computes the 75th, 95th, and 99th percentiles of the \"Latency\" column using a builder pattern, and the 95th\n and 99th percentiles of the \"Size\" column by \"Sym\":\n \n \n new ApproximatePercentile.PercentileDefinition(\"Latency\").add(0.75, \"L75\").add(0.95, \"L95\").add(0.99, \"L99\")\n .nextColumn(\"Size\").add(0.95, \"S95\").add(0.99, \"S99\");\n final Table aggregated = ApproximatePercentile.approximatePercentiles(input, definition);\n \n\n When parallelizing a workload, you may want to divide it based on natural partitioning and then compute an overall\n percentile. In these cases, you should use the ApproximatePercentile.PercentileDefinition.exposeDigest(java.lang.String) method to expose the\n internal t-digest structure as a column. If you then perform an array aggregation (Table.groupBy(java.util.Collection)), you can\n call the accumulateDigests(io.deephaven.vector.ObjectVector) function to produce a single digest that represents all of the constituent\n digests. The amount of error introduced is related to the compression factor that you have selected for the digests.\n Once you have a combined digest object, you can call the quantile or other functions to extract the desired\n percentile.", + "text": "Generate approximate percentile aggregations of a table.\n\n \n The underlying data structure and algorithm used is a t-digest as described at https://github.com/tdunning/t-digest,\n which has a \"compression\" parameter that determines the size of the retained values. From the t-digest documentation,\n \"100 is a common value for normal uses. 1000 is extremely large. The number of centroids retained will be a\n smallish (usually less than 10) multiple of this number."e;\n \n\n All input columns are cast to doubles and the result columns are doubles.\n \n\n The input table must be add only, if modifications or removals take place; then an UnsupportedOperationException is\n thrown. For tables with adds and removals you must use exact percentiles with Aggregation.AggPct(double, java.lang.String...).\n \n\n You may compute either one approximate percentile or several approximate percentiles at once. For example, to compute\n the 95th percentile of all other columns, by the \"Sym\" column you would call:\n \n \n ApproximatePercentile.approximatePercentile(input, 0.95, \"Sym\")\n \n\n If you need to compute several percentiles, it is more efficient to compute them simultaneously. For example, this\n example computes the 75th, 95th, and 99th percentiles of the \"Latency\" column using a builder pattern, and the 95th\n and 99th percentiles of the \"Size\" column by \"Sym\":\n \n \n final Table aggregated = input.aggBy(List.of(\n Aggregation.ApproxPct(\"Latency', PctOut(0.75, \"L75\"), PctOut(0.95, \"L95\"), PctOut(0.99, \"L99\")\n Aggregation.ApproxPct(\"Size', PctOut(0.95, \"S95\"), PctOut(0.99, \"S99\")));\n \n\n When parallelizing a workload, you may want to divide it based on natural partitioning and then compute an overall\n percentile. In these cases, you should use the Aggregation.AggTDigest(java.lang.String...) aggregation to expose the internal\n t-digest structure as a column. If you then perform an array aggregation (Table.groupBy(java.util.Collection)), you can call the\n accumulateDigests(io.deephaven.vector.ObjectVector) function to produce a single digest that represents all of the constituent digests. The\n amount of error introduced is related to the compression factor that you have selected for the digests. Once you have\n a combined digest object, you can call the quantile or other functions to extract the desired percentile.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile/PercentileDefinition.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile/PercentileDefinition.json deleted file mode 100644 index fc96425177b..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ApproximatePercentile/PercentileDefinition.json +++ /dev/null @@ -1,12 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.ApproximatePercentile$PercentileDefinition", - "methods": { - "add": "Adds an output column.\n\n To set the inputColumn call nextColumn(String).\n\n:param percentile: (double) - the percentile to calculate\n:param resultName: (java.lang.String) - the result name\n:return: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) a (possibly new) PercentileDefinition", - "exposeDigest": "If true, the tDigest column is exposed using the given name\n\n:param digestColumnName: (java.lang.String) - the name of the t-digest column in the output\n:return: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) a (possibly new) PercentileDefinition", - "nextColumn": "Sets the name of the inputColumn\n\n:param inputColumn: (java.lang.String) - the name of the input column that subsequent calls to add(double, java.lang.String) operate on.\n:return: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) a (possibly new) PercentileDefinition", - "setCompression": "Sets the t-digest compression parameter.\n\n:param compression: (double) - the t-digest compression factor.\n:return: (io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition) a (possibly new) PercentileDefinition" - }, - "path": "io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition", - "text": "A builder class for an approximate percentile definition to be used with ApproximatePercentile.approximatePercentiles(io.deephaven.engine.table.Table, io.deephaven.engine.table.impl.by.ApproximatePercentile.PercentileDefinition, io.deephaven.engine.table.impl.select.SelectColumn...).", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AvgSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AvgSpec.json deleted file mode 100644 index 24d956e2374..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AvgSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AvgSpec", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.AvgSpec", - "text": "Factory for iterative average aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.json index ba6a208afd0..4114bc65a43 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.ByteStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.json index 71c2079c282..a57e2366326 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CharStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.CharStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountAggregationFactory.json deleted file mode 100644 index 9e397786bee..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.CountAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.CountAggregationFactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountBySpecImpl.json deleted file mode 100644 index 8b7f33073a8..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountBySpecImpl.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.CountBySpecImpl", - "methods": { - "getCountName": ":return: java.lang.String", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized." - }, - "path": "io.deephaven.engine.table.impl.by.CountBySpecImpl", - "text": "Aggregation state factory for countBy() operation.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountDistinctSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountDistinctSpec.json deleted file mode 100644 index ca515a958ec..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/CountDistinctSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.CountDistinctSpec", - "methods": { - "countNulls": ":return: boolean", - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized." - }, - "path": "io.deephaven.engine.table.impl.by.CountDistinctSpec", - "text": "An Iterative state factory that computes the count of distinct items within a particular state.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DistinctSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DistinctSpec.json deleted file mode 100644 index 3adcefa202d..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DistinctSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.DistinctSpec", - "methods": { - "countNulls": ":return: boolean", - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized." - }, - "path": "io.deephaven.engine.table.impl.by.DistinctSpec", - "text": "An Iterative state factory that computes the count of distinct items within a particular state. It supports rollups,\n and converts itself into a Sum at the second level of the rollup.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.json index 107f1b1a160..3c70b67de45 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.DoubleStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstBySpecImpl.json deleted file mode 100644 index 83569dd5a0b..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstBySpecImpl.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.FirstBySpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.FirstBySpecImpl", - "text": "State factory for firstBy using an InterativeIndexState to create a RowRedirection.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.json index 86dfbf599b7..0c98d08a755 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.FloatStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.json index 37d8fd43097..8ca9e5ec540 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.json @@ -8,7 +8,7 @@ "initializeRefreshing": "Initialize refreshing result support for this operator. As a side effect, make a factory method for converting\n upstream modified column sets to result modified column sets, to be invoked whenever this operator reports a\n modification in order to determine the operator's contribution to the final result modified column set.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization\n:param aggregationUpdateListener: (io.deephaven.engine.liveness.LivenessReferent) - The aggregation update listener, which may be needed for referential integrity\n:return: (java.util.function.UnaryOperator) A factory that produces a result modified column set from the upstream modified column set", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param previousValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param newValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param previousValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param newValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", "modifyRowKeys": "Called with the modified row keys when IterativeChunkedAggregationOperator.requiresRowKeys() returns true if our input columns have not\n changed (or we have none).\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param rowKeys: (io.deephaven.chunk.LongChunk) - the modified row keys for a given destination, in post-shift space\n :param destination: (long) - the destination that was modified\n :return: (boolean) true if the result should be considered modified", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRowKeys": "Whether the operator requires row keys. This implies that the operator must process shifts (i.e.\n IterativeChunkedAggregationOperator.shiftChunk(io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext, io.deephaven.chunk.Chunk, io.deephaven.chunk.Chunk, io.deephaven.chunk.LongChunk, io.deephaven.chunk.LongChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.WritableBooleanChunk)), and must observe modifications even when its input columns (if any) are not modified (i.e.\n IterativeChunkedAggregationOperator.modifyRowKeys(io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext, io.deephaven.chunk.LongChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.WritableBooleanChunk)).\n\n:return: (boolean) true if the operator requires row keys, false otherwise", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.json index 6b275b5bb4e..24df184f0d5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.IntStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.json index 25f5991061a..c03723c7292 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.json @@ -12,7 +12,7 @@ "modifyRowKeys": "Called with the modified row keys when requiresRowKeys() returns true if our input columns have not\n changed (or we have none).\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param rowKeys: (io.deephaven.chunk.LongChunk) - the modified row keys for a given destination, in post-shift space\n :param destination: (long) - the destination that was modified\n :return: (boolean) true if the result should be considered modified", "propagateFailure": "Called on error to propagate listener failure to this operator.\n\n:param originalException: (java.lang.Throwable) - The error Throwable\n:param sourceEntry: (io.deephaven.engine.table.TableListener.Entry) - The UpdatePerformanceTracker.Entry for the failed listener", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRowKeys": "Whether the operator requires row keys. This implies that the operator must process shifts (i.e.\n shiftChunk(io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext, io.deephaven.chunk.Chunk, io.deephaven.chunk.Chunk, io.deephaven.chunk.LongChunk, io.deephaven.chunk.LongChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.WritableBooleanChunk)), and must observe modifications even when its input columns (if any) are not modified (i.e.\n modifyRowKeys(io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext, io.deephaven.chunk.LongChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.WritableBooleanChunk)).\n\n:return: (boolean) true if the operator requires row keys, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeIndexSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeIndexSpec.json deleted file mode 100644 index d05e61d89ef..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeIndexSpec.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.IterativeIndexSpec", - "methods": {}, - "path": "io.deephaven.engine.table.impl.by.IterativeIndexSpec", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.json deleted file mode 100644 index d60cfd5ba74..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IterativeOperatorSpec.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.IterativeOperatorSpec", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator" - }, - "path": "io.deephaven.engine.table.impl.by.IterativeOperatorSpec", - "text": "Creates iterative operators for the supplied type.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.json deleted file mode 100644 index 366a75c493e..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyFirstOrLastBySpec.json +++ /dev/null @@ -1,12 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.KeyOnlyFirstOrLastBySpec", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "getResultColumn": ":return: java.lang.String", - "isLast": ":return: boolean", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.KeyOnlyFirstOrLastBySpec", - "text": "A Flavor of FirstBy that produces no values from the original table, only a named column of source keys.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LastBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LastBySpecImpl.json deleted file mode 100644 index 9cf5c2b7ac2..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LastBySpecImpl.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.LastBySpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.LastBySpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.json index 4d5984b3c52..8344ad0e63c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.LongStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.json deleted file mode 100644 index 941d83e9b53..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxBySpecImpl.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.MinMaxBySpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "isMinimum": ":return: boolean", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.MinMaxBySpecImpl", - "text": "Minimum and Maximum aggregation factory.\n\n Operates in two modes, for non-refreshing tables it requires very little state (just the current minimum or maximum).\n\n For refreshing tables, it requires maintaining a TreeMap of values to counts; so that if the min/max value is removed\n we are able to identify the next lowest/highest value.\n\n You can use AddOnlyMinMaxBySpecImpl if you want to force add-only behavior.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.json deleted file mode 100644 index 3149cf08ee9..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/MinMaxIterativeOperatorFactory.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.MinMaxIterativeOperatorFactory", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.MinMaxIterativeOperatorFactory", - "text": "Factory for iterative sum aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.json deleted file mode 100644 index 7dc86a5f927..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/NonKeyColumnAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.NonKeyColumnAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.NonKeyColumnAggregationFactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.json index 9969a08a6bf..40baa637424 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.ObjectStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.json index b5097108a88..eb93b2b57b9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PartitionByChunkedOperator.json @@ -10,7 +10,7 @@ "modifyRowKeys": "Called with the modified row keys when IterativeChunkedAggregationOperator.requiresRowKeys() returns true if our input columns have not\n changed (or we have none).\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param rowKeys: (io.deephaven.chunk.LongChunk) - the modified row keys for a given destination, in post-shift space\n :param destination: (long) - the destination that was modified\n :return: (boolean) true if the result should be considered modified", "propagateFailure": "Called on error to propagate listener failure to this operator.\n\n:param originalException: (java.lang.Throwable) - The error Throwable\n:param sourceEntry: (io.deephaven.engine.table.TableListener.Entry) - The UpdatePerformanceTracker.Entry for the failed listener", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRowKeys": "Whether the operator requires row keys. This implies that the operator must process shifts (i.e.\n IterativeChunkedAggregationOperator.shiftChunk(io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext, io.deephaven.chunk.Chunk, io.deephaven.chunk.Chunk, io.deephaven.chunk.LongChunk, io.deephaven.chunk.LongChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.WritableBooleanChunk)), and must observe modifications even when its input columns (if any) are not modified (i.e.\n IterativeChunkedAggregationOperator.modifyRowKeys(io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext, io.deephaven.chunk.LongChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.IntChunk, io.deephaven.chunk.WritableBooleanChunk)).\n\n:return: (boolean) true if the operator requires row keys, false otherwise", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.json deleted file mode 100644 index 0f2d74a442d..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileBySpecImpl.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.PercentileBySpecImpl", - "methods": { - "getAverageMedian": ":return: boolean", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "getPercentile": ":return: double" - }, - "path": "io.deephaven.engine.table.impl.by.PercentileBySpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.json deleted file mode 100644 index f6f845c7ac8..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/PercentileIterativeOperatorFactory.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.PercentileIterativeOperatorFactory", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.PercentileIterativeOperatorFactory", - "text": "Factory for iterative sum aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.json deleted file mode 100644 index 42eac6e06dd..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregatableStatefactory.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.ReaggregatableStatefactory", - "methods": {}, - "path": "io.deephaven.engine.table.impl.by.ReaggregatableStatefactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.json deleted file mode 100644 index 3d0253bd526..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ReaggregationIterativeOperator.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.ReaggregationIterativeOperator", - "methods": { - "currentValue": ":param state: ReaggregationIterativeOperator.STATE\n:return: ReaggregationIterativeOperator.T", - "getFinalResultType": ":return: java.lang.Class", - "getStateType": ":return: java.lang.Class", - "prev": ":param state: ReaggregationIterativeOperator.STATE\n:return: ReaggregationIterativeOperator.T" - }, - "path": "io.deephaven.engine.table.impl.by.ReaggregationIterativeOperator", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RollupConstants.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RollupConstants.json new file mode 100644 index 00000000000..4c55dd33b5f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RollupConstants.json @@ -0,0 +1,7 @@ +{ + "className": "io.deephaven.engine.table.impl.by.RollupConstants", + "methods": {}, + "path": "io.deephaven.engine.table.impl.by.RollupConstants", + "text": "Implementation constants for Table.rollup(java.util.Collection, java.util.Collection) support.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RowStatus.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RowStatus.json deleted file mode 100644 index 9ebfbc45121..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/RowStatus.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.RowStatus", - "methods": { - "valueOf": "Returns the enum constant of this type with the specified name.\nThe string must match exactly an identifier used to declare an\nenum constant in this type. (Extraneous whitespace characters are \nnot permitted.)\n\n:param name: (java.lang.String) - the name of the enum constant to be returned.\n:return: (io.deephaven.engine.table.impl.by.RowStatus) the enum constant with the specified name", - "values": "Returns an array containing the constants of this enum type, in\nthe order they are declared.\n\n:return: (io.deephaven.engine.table.impl.by.RowStatus[]) an array containing the constants of this enum type, in the order they are declared" - }, - "path": "io.deephaven.engine.table.impl.by.RowStatus", - "typeName": "enum" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.json deleted file mode 100644 index 015ae85c73c..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SelectDistinctSpecImpl.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.SelectDistinctSpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.SelectDistinctSpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.json index bfa2260bdde..eb147184653 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update" }, "path": "io.deephaven.engine.table.impl.by.ShortStreamSortedFirstOrLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstBy.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstBy.json deleted file mode 100644 index 35e5fb11d96..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstBy.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.SortedFirstBy", - "methods": {}, - "path": "io.deephaven.engine.table.impl.by.SortedFirstBy", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.json deleted file mode 100644 index c2b4076d2c0..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.SortedFirstOrLastByAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.SortedFirstOrLastByAggregationFactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.json deleted file mode 100644 index 3bf97dd87cc..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedFirstOrLastByFactoryImpl.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.SortedFirstOrLastByFactoryImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "getSortColumnNames": ":return: java.lang.String[]", - "isSortedFirst": ":return: boolean" - }, - "path": "io.deephaven.engine.table.impl.by.SortedFirstOrLastByFactoryImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedLastBy.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedLastBy.json deleted file mode 100644 index 10574aee401..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SortedLastBy.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.SortedLastBy", - "methods": {}, - "path": "io.deephaven.engine.table.impl.by.SortedLastBy", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StdSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StdSpec.json deleted file mode 100644 index 07a38eb1e62..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StdSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.StdSpec", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.StdSpec", - "text": "Factory for iterative standard deviation aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamFirstChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamFirstChunkedOperator.json index 360034faccc..611ca2671c5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamFirstChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamFirstChunkedOperator.json @@ -6,7 +6,7 @@ "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled.", "unchunkedRowSet": "Whether the operator can deal with an unchunked RowSet more efficiently than a chunked RowSet.\n\n:return: (boolean) true if the operator can deal with unchunked RowSets, false otherwise" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamLastChunkedOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamLastChunkedOperator.json index 15e27484ec1..887d2a0f13c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamLastChunkedOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StreamLastChunkedOperator.json @@ -4,7 +4,7 @@ "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "addRowSet": ":param context: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext\n:param rowSet: io.deephaven.engine.rowset.RowSet\n:param destination: long\n:return: boolean", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "unchunkedRowSet": "Whether the operator can deal with an unchunked RowSet more efficiently than a chunked RowSet.\n\n:return: (boolean) true if the operator can deal with unchunked RowSets, false otherwise" }, "path": "io.deephaven.engine.table.impl.by.StreamLastChunkedOperator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SumSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SumSpec.json deleted file mode 100644 index 35891ea4dc9..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/SumSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.SumSpec", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.SumSpec", - "text": "Factory for iterative sum aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.json index b81cc5c3f14..24d284081ab 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.json @@ -2,17 +2,20 @@ "className": "io.deephaven.engine.table.impl.by.TDigestPercentileOperator", "methods": { "addChunk": "Aggregate a chunk of data into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to aggregate into, parallel with startPositions and length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the addition\n :param values: (io.deephaven.chunk.Chunk) - the values to aggregate\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", + "compression": ":return: double", "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "getResultColumns": "Return a map of result columns produced by this operator.\n\n:return: (java.util.Map>) a map of name to columns for the result table", "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", + "makeSecondaryOperator": ":param percentile: double\n:param resultName: java.lang.String\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", - "modifyChunk": "Modify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n:param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n:param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n:param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n:param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n:param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n:param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n:param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n:param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified", + "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param previousValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param newValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", "propagateInitialState": "Perform any internal state keeping needed for destinations that were added during initialization.\n\n:param resultTable: (io.deephaven.engine.table.impl.QueryTable) - The result QueryTable after initialization", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", + "resetForStep": "Reset any per-step internal state. Note that the arguments to this method should not be mutated in any way.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - The upstream ShiftAwareListener.Update", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, "path": "io.deephaven.engine.table.impl.by.TDigestPercentileOperator", - "text": "Iterative average operator.", + "text": "Iterative T-Digest and percentile operator.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.json deleted file mode 100644 index e226071070c..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingFirstBySpecImpl.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.TrackingFirstBySpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.TrackingFirstBySpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.json deleted file mode 100644 index d1fa45ebf50..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/TrackingLastBySpecImpl.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.TrackingLastBySpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.TrackingLastBySpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/UniqueSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/UniqueSpec.json deleted file mode 100644 index 96ca91c7c06..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/UniqueSpec.json +++ /dev/null @@ -1,13 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.UniqueSpec", - "methods": { - "countNulls": ":return: boolean", - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "getNoKeyValue": ":return: java.lang.Object", - "getNonUniqueValue": ":return: java.lang.Object" - }, - "path": "io.deephaven.engine.table.impl.by.UniqueSpec", - "text": "An Iterative state factory that displays the singular unique value of the items within a particular state, or default\n values if none are present, or the values are not unique.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/VarSpec.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/VarSpec.json deleted file mode 100644 index acb2bad985a..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/VarSpec.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.VarSpec", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.VarSpec", - "text": "Factory for iterative variance aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.json deleted file mode 100644 index f2d8f2969df..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSpecImpl.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.WeightedAverageSpecImpl", - "methods": { - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "getWeightName": ":return: java.lang.String", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.WeightedAverageSpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.json deleted file mode 100644 index 3d7dcf7c6c4..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedSumSpecImpl.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.WeightedSumSpecImpl", - "methods": { - "getChunkedOperator": ":param type: java.lang.Class\n:param name: java.lang.String\n:param exposeInternalColumns: boolean\n:return: io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator", - "getMemoKey": "Produces a MemoKey for this AggregationSpec.\n\n \n If two AggregationSpecs have equal memoKeys, then Table.groupBy(java.util.Collection) operations that have the same group by\n columns may be memoized. In that case instead of recomputing the result; the original result will be used.\n \n\n If null is returned, the operation will not be memoized.\n \n\n:return: (io.deephaven.engine.table.impl.by.AggregationMemoKey) an AggregationMemoKey, null if this operation can not be memoized.", - "getWeightName": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.WeightedSumSpecImpl", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns.json new file mode 100644 index 00000000000..07df511db51 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns.json @@ -0,0 +1,13 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.NullColumns", + "methods": { + "builder": ":return: io.deephaven.engine.table.impl.by.rollup.NullColumns.Builder", + "from": ":param resultColumns: java.util.Map>\n:return: io.deephaven.engine.table.impl.by.rollup.NullColumns", + "of": ":param name: java.lang.String\n:param type: java.lang.Class\n:return: io.deephaven.engine.table.impl.by.rollup.NullColumns", + "resultColumns": ":return: java.util.Map>", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.NullColumns", + "text": "RollupAggregation that allows columns to be nulled-out at higher aggregation levels.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns/Builder.json new file mode 100644 index 00000000000..a4aaf7e5bb0 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/NullColumns/Builder.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.NullColumns$Builder", + "methods": { + "build": ":return: io.deephaven.engine.table.impl.by.rollup.NullColumns", + "putAllResultColumns": ":param resultColumns: java.util.Map>\n:return: io.deephaven.engine.table.impl.by.rollup.NullColumns.Builder", + "putResultColumns": ":param name: java.lang.String\n:param type: java.lang.Class\n:return: io.deephaven.engine.table.impl.by.rollup.NullColumns.Builder" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.NullColumns.Builder", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/Partition.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/Partition.json new file mode 100644 index 00000000000..aee7e3a8512 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/Partition.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.Partition", + "methods": { + "includeConstituents": ":return: boolean", + "of": ":param includeConstituents: boolean\n:return: io.deephaven.engine.table.impl.by.rollup.Partition", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.Partition", + "text": "RollupAggregation that represents a partitioning of the input table.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json new file mode 100644 index 00000000000..f1dc6201438 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.RollupAggregation", + "methods": { + "nullColumns": ":param resultColumns: java.util.Map>\n:return: io.deephaven.engine.table.impl.by.rollup.RollupAggregation", + "partition": ":param includeConstituents: boolean\n:return: io.deephaven.engine.table.impl.by.rollup.RollupAggregation", + "walk": "*Overload 1* \n Note: Java generics information - \n \n :param visitor: V\n :return: V\n \n*Overload 2* \n Note: Java generics information - \n \n :param visitor: V\n :return: V" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.RollupAggregation", + "text": "Rollup-specific aggregations.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation/Visitor.json new file mode 100644 index 00000000000..3e456a51e08 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation/Visitor.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.RollupAggregation$Visitor", + "methods": { + "visit": "*Overload 1* \n :param nullColumns: io.deephaven.engine.table.impl.by.rollup.NullColumns\n \n*Overload 2* \n :param partition: io.deephaven.engine.table.impl.by.rollup.Partition" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.RollupAggregation.Visitor", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json new file mode 100644 index 00000000000..14e57a783c1 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.RollupAggregationBase", + "methods": { + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.RollupAggregationBase", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.json index cd0381b3560..0055be8cf8d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.json @@ -3,11 +3,11 @@ "methods": { "checkNumericCompatibility": ":param value: java.lang.Number\n:param expected: java.lang.Class\n:return: boolean", "checkType": ":param resultColName: java.lang.String\n:param valueIntent: java.lang.String\n:param expected: java.lang.Class\n:param value: java.lang.Object", - "createCountDistinct": "Create an operator for the\n AggType.CountDistinct aggregation.\n\n:param type: (java.lang.Class) - the type of the column\n:param resultName: (java.lang.String) - the name of the result column\n:param countNulls: (boolean) - true if null values should be counted as important values, or false if they should be ignored.\n:param exposeInternal: (boolean) - true if the underlying SSM state should be exposed as a column (for use with rollups)\n:param isRollup: (boolean) - true if the returned operator should be suitable for second or higher levels of rollup\n aggregation\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator) an appropriate operator.", - "createDistinct": "Create an operator for the\n AggType.Distinct aggregation.\n\n:param type: (java.lang.Class) - the type of the column\n:param resultName: (java.lang.String) - the name of the result column\n:param countNulls: (boolean) - true if null values should be counted as important values, or false if they should be ignored.\n:param exposeInternal: (boolean) - true if the underlying SSM state should be exposed as a column (for use with rollups)\n:param isRollup: (boolean) - true if the returned operator should be suitable for second or higher levels of rollup\n aggregation\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator) an appropriate operator.", - "createUnique": "Create an operator for the\n AggType.Unique aggregation.\n\n:param type: (java.lang.Class) - the type of the column\n:param resultName: (java.lang.String) - the name of the result column\n:param countNulls: (boolean) - true if null values should be counted as important values, or false if they should be ignored.\n:param exposeInternal: (boolean) - true if the underlying SSM state should be exposed as a column (for use with rollups)\n:param noKeyValue: java.lang.Object\n:param nonUniqueValue: java.lang.Object\n:param isRollup: (boolean) - true if the returned operator should be suitable for second or higher levels of rollup\n aggregation\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator) an appropriate operator." + "createCountDistinct": "Create an operator for the\n io.deephaven.engine.table.impl.by.AggType#CountDistinct aggregation.\n\n:param type: (java.lang.Class) - the type of the column\n:param resultName: (java.lang.String) - the name of the result column\n:param countNulls: (boolean) - true if null values should be counted as important values, or false if they should be ignored.\n:param exposeInternal: (boolean) - true if the underlying SSM state should be exposed as a column (for use with rollups)\n:param isRollup: (boolean) - true if the returned operator should be suitable for second or higher levels of rollup\n aggregation\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator) an appropriate operator.", + "createDistinct": "Create an operator for the\n io.deephaven.engine.table.impl.by.AggType#Distinct aggregation.\n\n:param type: (java.lang.Class) - the type of the column\n:param resultName: (java.lang.String) - the name of the result column\n:param countNulls: (boolean) - true if null values should be counted as important values, or false if they should be ignored.\n:param exposeInternal: (boolean) - true if the underlying SSM state should be exposed as a column (for use with rollups)\n:param isRollup: (boolean) - true if the returned operator should be suitable for second or higher levels of rollup\n aggregation\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator) an appropriate operator.", + "createUnique": "Create an operator for the\n io.deephaven.engine.table.impl.by.AggType#Unique aggregation.\n\n:param type: (java.lang.Class) - the type of the column\n:param resultName: (java.lang.String) - the name of the result column\n:param countNulls: (boolean) - true if null values should be counted as important values, or false if they should be ignored.\n:param onlyNullsSentinel: java.lang.Object\n:param nonUniqueSentinel: java.lang.Object\n:param exposeInternal: (boolean) - true if the underlying SSM state should be exposed as a column (for use with rollups)\n:param isRollup: (boolean) - true if the returned operator should be suitable for second or higher levels of rollup\n aggregation\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator) an appropriate operator." }, "path": "io.deephaven.engine.table.impl.by.ssmcountdistinct.DistinctOperatorFactory", - "text": "A factory interface to create operators for the\n AggType.Distinct\nAggType.CountDistinct,\n AggType.Unique and their rollup counterparts", + "text": "A factory interface to create operators for the\n io.deephaven.engine.table.impl.by.AggType#Distinct\nio.deephaven.engine.table.impl.by.AggType#CountDistinct,\n io.deephaven.engine.table.impl.by.AggType#Unique and their rollup counterparts", "typeName": "interface" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.json index e8adca2e356..32ae68ab1d5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.json index 7a4b2c6a898..46eaec298e0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ByteRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.json index 12f3ea472ca..8941832fd69 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.json index a29285ca98b..50b48b87b5f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/CharRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.json index 4e009e0f255..90119599589 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.json index 1f10bb232f8..fa1443cce6d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/DoubleRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.json index 83abd3aeb45..c1d6ad3f542 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.json index 837bc278372..545e3dd65c9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/FloatRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.json index 5d4daf5b6ab..c2f5f07b71e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.json index 1c902cf9774..f6732c6a114 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/IntRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.json index fb6a8617fbb..939694affcf 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.json index a3a23685091..b564ffedebc 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/LongRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.json index 9420366a9c0..457b91e5ee6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.json index 004a4ffa6cb..a16b4165e1e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ObjectRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.json index c48c50d744b..0a0f95fe38c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortChunkedCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.json index bd2eaa16b88..b9e854212e2 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/count/ShortRollupCountDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "requiresRunFinds": "Some operators require that all values for a destination within a given chunk are adjacent. These operators must\n return true.\n\n:return: (boolean) true if the operator requires runs of the same destination to be adjacent", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.json index da3a2165c5d..7a4dae411a5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.json index 09723798a72..70d16b43ef2 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ByteRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.json index 1f56222cf90..fc1ffcac927 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.json index e3b7a83f0a6..b915d71e60c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/CharRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.json index 226300fdb7d..b9758ee26fe 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.json index 53cc2cc5d7f..1df0362b715 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/DoubleRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.json index 6a4f7f395a9..c1ca4cba1a9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.json index 2c70f95b3ee..bbdec1559cf 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/FloatRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.json index 31b3398029d..3f5b1a42d4c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.json index a0a31ad3bc9..3f847226fe4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/IntRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.json index 72eb6bb61fd..b0a81aab9ba 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.json index bb4e7c91687..d8dab0a5224 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.json index aa753c3d48f..4c54f8925a0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.json index 2e6b9028d14..fea250cb12c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ObjectRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.json index c34ad338a0a..7e32334b9d3 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortChunkedDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.json index 1302f460725..006f19ea71b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/ShortRollupDistinctOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.json index 555ec2af31d..6a01ffeb12c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.json index c6874e6d8bc..1b094cb7206 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ByteRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.json index a963a71c978..535cd895cad 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.json index b709c9a2a21..89c55660520 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/CharRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.json index 275f3fa3281..cea80f9bccd 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.json index a2c4c84bcaa..09e935c5a22 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/DoubleRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.json index 5524a1cb0c9..0ff080f667a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.json index eecda621eaf..1db15f2b54e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/FloatRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.json index e8454b119c6..c423d692200 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.json index 9ce703083f0..aa0437fe459 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/IntRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.json index 66809e56852..81d41417bf4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.json index 26054fc86d9..90de7435d17 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.json index afd803c2cde..b5e2dbcbbfb 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.json index d16f77df478..207b3648d5b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ObjectRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.json index 7d319d6cf5d..6d741a07db6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortChunkedUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.json index 87aa5d4bfd6..0a1e218bf3f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/ShortRollupUniqueOperator.json @@ -7,7 +7,7 @@ "makeBucketedContext": "Make a IterativeChunkedAggregationOperator.BucketedContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) A new IterativeChunkedAggregationOperator.BucketedContext, or null if none is necessary", "makeSingletonContext": "Make a IterativeChunkedAggregationOperator.SingletonContext suitable for this operator if necessary.\n\n:param size: (int) - The maximum size of input chunks that will be used with the result context\n:return: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) A new IterativeChunkedAggregationOperator.SingletonContext, or null if none is necessary", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param preValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param postValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param bucketedContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param singletonContext: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/FillContext.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/FillContext.json new file mode 100644 index 00000000000..da98fc5092c --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/FillContext.json @@ -0,0 +1,6 @@ +{ + "className": "io.deephaven.engine.table.impl.select.Formula$FillContext", + "methods": {}, + "path": "io.deephaven.engine.table.impl.select.Formula.FillContext", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/GetContext.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/GetContext.json new file mode 100644 index 00000000000..0c58fd46608 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/formula/GetContext.json @@ -0,0 +1,6 @@ +{ + "className": "io.deephaven.engine.table.impl.select.Formula$GetContext", + "methods": {}, + "path": "io.deephaven.engine.table.impl.select.Formula.GetContext", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FillFromUnordered.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FillFromUnordered.json deleted file mode 100644 index cecaf88f90d..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FillFromUnordered.json +++ /dev/null @@ -1,8 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.sources.FillFromUnordered", - "methods": { - "fillFromChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched" - }, - "path": "io.deephaven.engine.table.impl.sources.FillFromUnordered", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableBooleanArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableBooleanArraySource.json deleted file mode 100644 index 5259de19dbc..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableBooleanArraySource.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableBooleanArraySource", - "methods": { - "get": ":param index: long\n:return: java.lang.Boolean", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" - }, - "path": "io.deephaven.engine.table.impl.sources.immutable.ImmutableBooleanArraySource", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDateTimeArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDateTimeArraySource.json deleted file mode 100644 index 3fd24fdd1b0..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDateTimeArraySource.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableDateTimeArraySource", - "methods": { - "get": ":param index: long\n:return: io.deephaven.time.DateTime", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" - }, - "path": "io.deephaven.engine.table.impl.sources.immutable.ImmutableDateTimeArraySource", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.json index ddbbb42c4d2..7f9f85e3030 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/tuplesource/TupleSourceCreatorImpl.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.engine.table.impl.tuplesource.TupleSourceCreatorImpl", "methods": { - "makeTupleSource": "See TupleSourceFactory.makeTupleSource(ColumnSource[]).\n\n:param columnSources: io.deephaven.engine.table.ColumnSource...\n:return: io.deephaven.engine.table.TupleSource" + "makeTupleSource": "See TupleSourceFactory.makeTupleSource(ColumnSource[]).\n\nNote: Java generics information - \n\n:param columnSources: io.deephaven.engine.table.ColumnSource...\n:return: io.deephaven.engine.table.TupleSource" }, "path": "io.deephaven.engine.table.impl.tuplesource.TupleSourceCreatorImpl", "text": "Factory for TupleSource instances.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/freezeby/FreezeByOperator.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/freezeby/FreezeByOperator.json index ca5407bb2fe..5c0f6a112b4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/freezeby/FreezeByOperator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/freezeby/FreezeByOperator.json @@ -5,7 +5,7 @@ "ensureCapacity": "Ensure that this operator can handle destinations up to tableSize - 1.\n\n:param tableSize: (long) - the new size of the table", "getResultColumns": "Return a map of result columns produced by this operator.\n\n:return: (java.util.Map>) a map of name to columns for the result table", "modifyChunk": "**Incompatible overloads text - text from the first overload:**\n\nModify a chunk of data previously aggregated into the result columns using a parallel chunk of new values. Never\n includes modifies that have been shifted if IterativeChunkedAggregationOperator.requiresRowKeys() returns true - those are handled in\n IterativeChunkedAggregationOperator.shiftChunk(BucketedContext, Chunk, Chunk, LongChunk, LongChunk, IntChunk, IntChunk, IntChunk, WritableBooleanChunk).\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param previousValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated\n :param newValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the modification\n :param previousValues: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param newValues: (io.deephaven.chunk.Chunk) - a chunk of values to aggregate\n :param postShiftRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in post-shift space\n :param destination: long\n :return: (boolean) true if the state was modified, false otherwise", - "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream ShiftAwareListener.Update (which does not have its\n ModifiedColumnSet finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", + "propagateUpdates": "Perform any internal state keeping needed for destinations that were added (went from 0 keys to > 0), removed\n (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that\n the arguments to this method should not be mutated in any way.\n\n:param downstream: (io.deephaven.engine.table.TableUpdate) - The downstream TableUpdate (which does not have its ModifiedColumnSet\n finalized yet)\n:param newDestinations: (io.deephaven.engine.rowset.RowSet) - New destinations added on this update", "removeChunk": "Remove a chunk of data previously aggregated into the result columns.\n\n*Overload 1* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.BucketedContext) - the operator-specific context\n :param values: (io.deephaven.chunk.Chunk) - a chunk of values that have been previously aggregated.\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destinations: (io.deephaven.chunk.IntChunk) - the destinations in resultColumn to remove the values from, parallel with startPositions and\n length\n :param startPositions: (io.deephaven.chunk.IntChunk) - the starting positions in the chunk for each destination\n :param length: (io.deephaven.chunk.IntChunk) - the number of values in the chunk for each destination\n :param stateModified: (io.deephaven.chunk.WritableBooleanChunk) - a boolean output array, parallel to destinations, which is set to true if the corresponding\n destination has been modified\n \n*Overload 2* \n :param context: (io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.SingletonContext) - the operator-specific context\n :param chunkSize: (int) - the size of the removal\n :param values: (io.deephaven.chunk.Chunk) - the values to remove from the aggregation\n :param inputRowKeys: (io.deephaven.chunk.LongChunk) - the input row keys, in pre-shift space\n :param destination: (long) - the destination in the result columns\n :return: (boolean) true if the state was modified, false otherwise", "startTrackingPrevValues": "Called after initialization; when the operator's result columns must have previous tracking enabled." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder.json index e0ecbac58dc..49ef197e14e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder.json @@ -1,23 +1,23 @@ { "className": "io.deephaven.engine.util.TotalsTableBuilder", "methods": { - "addOperation": "**Incompatible overloads text - text from the first overload:**\n\nAdds an operation for a column.\n\n \n The add method is used instead of the setOperation(String, AggType, String) method when more than one\n aggregation per input column is desired.\n\n*Overload 1* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.table.impl.by.AggType) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.table.impl.by.AggType) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 3* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 4* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", + "addOperation": "**Incompatible overloads text - text from the first overload:**\n\nAdds an operation for a column.\n\n \n The add method is used instead of the setOperation(String, AggType, String) method when more than one\n aggregation per input column is desired.\n\n*Overload 1* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 3* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 4* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", "applyToTable": "Helper method for building and applying a totals table to a Table.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table\n:return: (io.deephaven.engine.table.Table) table.setTotalsTable(buildDirective())", "buildDirective": "Creates the string directive used to set the Table attribute.\n\n:return: (java.lang.String) the attribute string representing this TotalsTableBuilder.", "fromDirective": "Produce a TotalsTableBuilder from a directive string.\n\n \n The buildDirective() method produces a String representation of a TotalsTableBuilder, this function is\n its inverse.\n \n\n:param directive: (java.lang.String) - the directive.\n:return: (io.deephaven.engine.util.TotalsTableBuilder) a TotalsTableBuilder", "get": "Produce a totals table builder from the source table.\n\n:param source: (io.deephaven.engine.table.Table) - the source table\n:return: (io.deephaven.engine.util.TotalsTableBuilder) a TotalsTableBuilder derived from the source table's totals table attribute", - "getDefaultOperation": "Gets the operation to use for columns without an operation specified.\n\n:return: (io.deephaven.engine.table.impl.by.AggType) the default operation", - "getFormats": "Gets the format for an aggregated column.\n\n:param column: (java.lang.String) - the column to get the format for\n:return: (java.util.Map) a map from AggType to the corresponding format string", - "getOperations": "Gets the operations for a given column.\n\n:param column: (java.lang.String) - the column to get the operations for\n:return: (java.util.Set) a set of aggregations for the column", + "getDefaultOperation": "Gets the operation to use for columns without an operation specified.\n\n:return: (io.deephaven.engine.util.TotalsTableBuilder.AggType) the default operation", + "getFormats": "Gets the format for an aggregated column.\n\n:param column: (java.lang.String) - the column to get the format for\n:return: (java.util.Map) a map from AggType to the corresponding format string", + "getOperations": "Gets the operations for a given column.\n\n:param column: (java.lang.String) - the column to get the operations for\n:return: (java.util.Set) a set of aggregations for the column", "getShowGrandTotalsByDefault": "Should grand totals be shown by default?\n\n:return: (boolean) true if grand totals should be shown by default", "getShowTotalsByDefault": "Should totals be shown by default?\n\n:return: (boolean) true if totals should be shown by default", "hasDefinedTotals": "Does a table have a totals table defined?\n\n:param source: (io.deephaven.engine.table.Table) - the source table\n:return: (boolean) true if source has a totals table defined", - "makeAggregationFactory": "Produce a AggregationFactory from a source table and builder.\n\n:param source: (io.deephaven.engine.table.Table) - the source table\n:param builder: (io.deephaven.engine.util.TotalsTableBuilder) - the TotalsTableBuilder\n:return: (io.deephaven.engine.table.impl.by.AggregationFactory) the AggregationFactory described by source and builder.", + "makeAggregations": "Produce aggregationsfrom a source table and builder.\n\n:param source: (io.deephaven.engine.table.Table) - the source table\n:param builder: (io.deephaven.engine.util.TotalsTableBuilder) - the TotalsTableBuilder\n:return: (java.util.Collection) the aggregations described by source and builder.", "makeTotalsTable": "**Incompatible overloads text - text from the first overload:**\n\nCreate a totals table from a source table.\n\n \n Given a source table that has had a TotalsTableBuilder applied, create a new totals table from the table. If no\n TotalsTableBuilder has been applied, then the columns are summed.\n\n*Overload 1* \n :param source: (io.deephaven.engine.table.Table) - the source table\n :return: (io.deephaven.engine.table.Table) an aggregated totals table\n \n*Overload 2* \n :param source: (io.deephaven.engine.table.Table) - the source table\n :param builder: (io.deephaven.engine.util.TotalsTableBuilder) - the TotalsTableBuilder\n :param groupByColumns: (java.lang.String...) - the columns to group by\n :return: (io.deephaven.engine.table.Table) an aggregated totals table", - "operationApplies": "Does the operation apply to type?\n\n:param type: (java.lang.Class) - the column type\n:param operation: (io.deephaven.engine.table.impl.by.AggType) - the aggregation operation\n:return: (boolean) true if the operation is applicable to columns of this type", - "setDefaultOperation": "Sets the operation for columns which are not otherwise specified.\n\n*Overload 1* \n :param defaultOperation: (io.deephaven.engine.table.impl.by.AggType) - the default operation\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param defaultOperation: (java.lang.String) - the default operation\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", - "setFormat": "Sets the format of a column.\n\n*Overload 1* \n :param column: (java.lang.String) - the column to set the format for\n :param agg: (io.deephaven.engine.table.impl.by.AggType) - the aggregation type the format is relevant for\n :param format: (java.lang.String) - the format string\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param column: (java.lang.String) - the column to set the format for\n :param agg: (java.lang.String) - the aggregation type the format is relevant for, \"*\" for all aggregations\n :param format: (java.lang.String) - the format string\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", - "setOperation": "Sets the operation for a column.\n\n*Overload 1* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.table.impl.by.AggType) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.table.impl.by.AggType) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 3* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 4* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", + "operationApplies": "Does the operation apply to type?\n\n:param type: (java.lang.Class) - the column type\n:param operation: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the aggregation operation\n:return: (boolean) true if the operation is applicable to columns of this type", + "setDefaultOperation": "Sets the operation for columns which are not otherwise specified.\n\n*Overload 1* \n :param defaultOperation: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the default operation\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param defaultOperation: (java.lang.String) - the default operation\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", + "setFormat": "Sets the format of a column.\n\n*Overload 1* \n :param column: (java.lang.String) - the column to set the format for\n :param agg: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the aggregation type the format is relevant for\n :param format: (java.lang.String) - the format string\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param column: (java.lang.String) - the column to set the format for\n :param agg: (java.lang.String) - the aggregation type the format is relevant for, \"*\" for all aggregations\n :param format: (java.lang.String) - the format string\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", + "setOperation": "Sets the operation for a column.\n\n*Overload 1* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 2* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (io.deephaven.engine.util.TotalsTableBuilder.AggType) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 3* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder\n \n*Overload 4* \n :param column: (java.lang.String) - the name of the column to operate on\n :param operation: (java.lang.String) - the aggregation operation for this column\n :param format: (java.lang.String) - the format string for this column\n :return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", "setShowGrandTotalsByDefault": "Set whether grand totals are shown by default.\n\n:param showGrandTotalsByDefault: (boolean) - whether grand totals are shown by default\n:return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", "setShowTotalsByDefault": "Set whether totals are shown by default.\n\n:param showTotalsByDefault: (boolean) - whether totals are shown by default\n:return: (io.deephaven.engine.util.TotalsTableBuilder) this TotalsTableBuilder", "toString": ":return: java.lang.String" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggType.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder/AggType.json similarity index 52% rename from Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggType.json rename to Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder/AggType.json index 2ca4bbaac69..3956f8648f6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggType.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/TotalsTableBuilder/AggType.json @@ -1,10 +1,10 @@ { - "className": "io.deephaven.engine.table.impl.by.AggType", + "className": "io.deephaven.engine.util.TotalsTableBuilder$AggType", "methods": { - "valueOf": "Returns the enum constant of this type with the specified name.\nThe string must match exactly an identifier used to declare an\nenum constant in this type. (Extraneous whitespace characters are \nnot permitted.)\n\n:param name: (java.lang.String) - the name of the enum constant to be returned.\n:return: (io.deephaven.engine.table.impl.by.AggType) the enum constant with the specified name", - "values": "Returns an array containing the constants of this enum type, in\nthe order they are declared.\n\n:return: (io.deephaven.engine.table.impl.by.AggType[]) an array containing the constants of this enum type, in the order they are declared" + "valueOf": "Returns the enum constant of this type with the specified name.\nThe string must match exactly an identifier used to declare an\nenum constant in this type. (Extraneous whitespace characters are \nnot permitted.)\n\n:param name: (java.lang.String) - the name of the enum constant to be returned.\n:return: (io.deephaven.engine.util.TotalsTableBuilder.AggType) the enum constant with the specified name", + "values": "Returns an array containing the constants of this enum type, in\nthe order they are declared.\n\n:return: (io.deephaven.engine.util.TotalsTableBuilder.AggType[]) an array containing the constants of this enum type, in the order they are declared" }, - "path": "io.deephaven.engine.table.impl.by.AggType", - "text": "Enumeration representing valid aggregation types for AggregationFactory or TotalsTableBuilder.", + "path": "io.deephaven.engine.util.TotalsTableBuilder.AggType", + "text": "Enumeration representing valid aggregation types for TotalsTableBuilder.", "typeName": "enum" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plot/util/PlotUtils.json b/Integrations/python/deephaven/doc/io/deephaven/plot/util/PlotUtils.json index d8ab700b5ca..7acd8995d5f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/plot/util/PlotUtils.json +++ b/Integrations/python/deephaven/doc/io/deephaven/plot/util/PlotUtils.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.plot.util.PlotUtils", "methods": { - "createCategoryComboAgg": ":param agg: io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement\n:return: io.deephaven.engine.table.impl.by.AggregationFactory", + "createCategoryAggs": ":param agg: io.deephaven.api.agg.Aggregation\n:return: java.util.Collection", "createCategoryHistogramTable": ":param t: io.deephaven.engine.table.Table\n:param byColumns: java.lang.String...\n:return: io.deephaven.engine.table.Table", "createCategoryTable": ":param t: io.deephaven.engine.table.Table\n:param catColumns: java.lang.String[]\n:return: io.deephaven.engine.table.Table", "createCategoryTableHandle": "*Overload 1* \n :param t: io.deephaven.engine.table.Table\n :param catColumn: java.lang.String\n :param otherColumns: java.lang.String...\n :return: io.deephaven.plot.util.tables.TableHandle\n \n*Overload 2* \n :param t: io.deephaven.engine.table.Table\n :param catColumns: java.lang.String[]\n :param otherColumns: java.lang.String...\n :return: io.deephaven.plot.util.tables.TableHandle", diff --git a/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java b/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java index 41799f332c5..0a74203b6b0 100644 --- a/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java +++ b/Util/util-immutables/src/main/java/io/deephaven/annotations/TupleStyle.java @@ -9,14 +9,14 @@ import java.lang.annotation.Target; /** - * A tuple style is for objects that represent simple tuples, with all parameters specified at construction. Not recommended for objects with more than two fields. Not - * applicable for objects with default fields. + * A tuple style is for objects that represent simple tuples, with all parameters specified at construction. Not + * recommended for objects with more than two fields. Not applicable for objects with default fields. */ @Target({ElementType.TYPE, ElementType.PACKAGE}) @Retention(RetentionPolicy.CLASS) @Value.Style(visibility = ImplementationVisibility.PACKAGE, allParameters = true, - typeImmutable = "*Tuple", + typeImmutable = "*Tuple", defaults = @Value.Immutable(builder = false, copy = false), strictBuilder = true, weakInterning = true, jdkOnly = true) public @interface TupleStyle { diff --git a/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java b/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java index e32cb10ab48..3cbad615fa8 100644 --- a/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java +++ b/engine/base/src/main/java/io/deephaven/engine/table/impl/TupleSourceFactory.java @@ -39,7 +39,7 @@ public interface TupleSourceCreator { * @param columnSources The column sources * @return The tuple factory */ - public static TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) { + public static TupleSource makeTupleSource(@NotNull final ColumnSource... columnSources) { return tupleSourceCreator().makeTupleSource(columnSources); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java index f3a29f242a7..00715401e2c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/MemoizedOperationKey.java @@ -96,7 +96,7 @@ public static MemoizedOperationKey treeTable(String idColumn, String parentColum } public static MemoizedOperationKey aggBy(Collection aggregations, - SelectColumn[] groupByColumns) { + SelectColumn[] groupByColumns) { if (!isMemoizable(groupByColumns)) { return null; } @@ -111,7 +111,7 @@ public static MemoizedOperationKey partitionBy(boolean dropKeys, SelectColumn[] } public static MemoizedOperationKey rollup(Collection aggregations, - SelectColumn[] groupByColumns, boolean includeConstituents) { + SelectColumn[] groupByColumns, boolean includeConstituents) { if (!isMemoizable(groupByColumns)) { return null; } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java index a2573aca5c6..799e6186149 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/QueryTable.java @@ -626,8 +626,8 @@ public Table aggBy( final List optimized = AggregationOptimizer.of(aggregations); final SelectColumn[] gbsColumns = SelectColumn.from(groupByColumns); final MemoizedOperationKey aggKey = MemoizedOperationKey.aggBy(optimized, gbsColumns); - final Table aggregationTable = memoizeResult(aggKey, () -> - aggNoMemo(AggregationProcessor.forAggregation(optimized), gbsColumns)); + final Table aggregationTable = + memoizeResult(aggKey, () -> aggNoMemo(AggregationProcessor.forAggregation(optimized), gbsColumns)); final List optimizedOrder = AggregationPairs.outputsOf(optimized).collect(Collectors.toList()); final List userOrder = AggregationPairs.outputsOf(aggregations).collect(Collectors.toList()); @@ -650,7 +650,7 @@ public Table countBy(String countColumnName, Selectable... groupByColumns) { } private QueryTable aggNoMemo(@NotNull final AggregationContextFactory aggregationContextFactory, - @NotNull final SelectColumn... groupByColumns) { + @NotNull final SelectColumn... groupByColumns) { final String description = "aggregation(" + aggregationContextFactory + ", " + Arrays.toString(groupByColumns) + ")"; return QueryPerformanceRecorder.withNugget(description, sizeForInstrumentation(), diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java index c2897ed5931..7ce8b9161e3 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationContext.java @@ -81,7 +81,7 @@ class AggregationContext { } AggregationContext(IterativeChunkedAggregationOperator[] operators, String[][] inputNames, - ChunkSource.WithPrev[] inputColumns, AggregationContextTransformer[] transformers) { + ChunkSource.WithPrev[] inputColumns, AggregationContextTransformer[] transformers) { this.operators = operators; this.inputNames = inputNames; this.inputColumns = inputColumns; @@ -263,8 +263,8 @@ void resetOperatorsForStep(@NotNull final TableUpdate upstream) { * keys to > 0), removed (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by * this iteration. Note that the arguments to this method should not be mutated in any way. * - * @param downstream The downstream {@link TableUpdate} (which does not have its - * {@link ModifiedColumnSet} finalized yet) + * @param downstream The downstream {@link TableUpdate} (which does not have its {@link ModifiedColumnSet} + * finalized yet) * @param newDestinations New destinations added on this update */ void propagateChangesToOperators(@NotNull final TableUpdate downstream, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.java index ce76a5c34ed..b11847fe5d9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ByteStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.java index b09ad168b5e..5d3e73ce14b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/DoubleStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.java index cf048476836..e87e95d8209 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FloatStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java index b6b58ca00a0..66237bd10df 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FpChunkedNonNormalCounter.java @@ -205,7 +205,8 @@ Map> fpInternalColumnSources(final String name) { if (nanCount != null) { results.put(name + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, nanCount); } else { - results.put(name + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, new WrappedLongArraySource(() -> nanCount)); + results.put(name + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, + new WrappedLongArraySource(() -> nanCount)); } if (positiveInfinityCount != null) { results.put(name + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX, positiveInfinityCount); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.java index b5a7dc3f89b..c0961644443 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IntStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java index 56f4174f372..6fc6187f726 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IterativeChunkedAggregationOperator.java @@ -312,8 +312,8 @@ default void resetForStep(@NotNull final TableUpdate upstream) {} * (went from > 0 keys to 0), or modified (keys added or removed, or keys modified) by this iteration. Note that * the arguments to this method should not be mutated in any way. * - * @param downstream The downstream TableUpdate (which does not have its - * {@link ModifiedColumnSet} finalized yet) + * @param downstream The downstream TableUpdate (which does not have its {@link ModifiedColumnSet} + * finalized yet) * @param newDestinations New destinations added on this update */ default void propagateUpdates(@NotNull final TableUpdate downstream, diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.java index 1d35960a596..3c7ef3bcc5a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/LongStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.java index 97c81fc56a2..4230d71b83f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ObjectStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java index c324c631dda..0d7e4a47bfc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RollupConstants.java @@ -5,8 +5,7 @@ */ public final class RollupConstants { - private RollupConstants() { - } + private RollupConstants() {} /** * Marker suffiix for rollup-internal column names. diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.java index 33a95c18400..2b20327013c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ShortStreamSortedFirstOrLastChunkedOperator.java @@ -144,7 +144,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { public void propagateUpdates(@NotNull TableUpdate downstream, @NotNull RowSet newDestinations) { Assert.assertion(downstream.removed().isEmpty() && downstream.shifted().empty(), "downstream.removed.empty() && downstream.shifted.empty()"); - // In a combo-agg, we may get modifications from other other operators that we didn't record as modifications in + // In a combo-agg, we may get modifications from other operators that we didn't record as modifications in // our redirections, so we separately track updated destinations. try (final RowSequence changedDestinations = isCombo ? changedDestinationsBuilder.build() : downstream.modified().union(downstream.added())) { copyStreamToResult(changedDestinations); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java index 5d17802f4b2..1f856473712 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/DistinctOperatorFactory.java @@ -120,8 +120,8 @@ static IterativeChunkedAggregationOperator createDistinct(Class type, String * @return an appropriate operator. */ static IterativeChunkedAggregationOperator createUnique(Class type, String resultName, boolean countNulls, - Object onlyNullsSentinel, Object nonUniqueSentinel, - boolean exposeInternal, boolean isRollup) { + Object onlyNullsSentinel, Object nonUniqueSentinel, + boolean exposeInternal, boolean isRollup) { checkType(resultName, "Only Nulls Sentinel", type, onlyNullsSentinel); checkType(resultName, "Non Unique Sentinel", type, nonUniqueSentinel); @@ -139,13 +139,17 @@ static IterativeChunkedAggregationOperator createUnique(Class type, String re io.deephaven.util.type.TypeUtils.unbox((Character) onlyNullsSentinel), io.deephaven.util.type.TypeUtils.unbox((Character) nonUniqueSentinel)); } else if (type == Double.class || type == double.class) { - final double onsAsType = (onlyNullsSentinel == null) ? NULL_DOUBLE : ((Number) onlyNullsSentinel).doubleValue(); - final double nusAsType = (nonUniqueSentinel == null) ? NULL_DOUBLE : ((Number) nonUniqueSentinel).doubleValue(); + final double onsAsType = + (onlyNullsSentinel == null) ? NULL_DOUBLE : ((Number) onlyNullsSentinel).doubleValue(); + final double nusAsType = + (nonUniqueSentinel == null) ? NULL_DOUBLE : ((Number) nonUniqueSentinel).doubleValue(); return isRollup ? new DoubleRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) : new DoubleChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Float.class || type == float.class) { - final float onsAsType = (onlyNullsSentinel == null) ? NULL_FLOAT : ((Number) onlyNullsSentinel).floatValue(); - final float nusAsType = (nonUniqueSentinel == null) ? NULL_FLOAT : ((Number) nonUniqueSentinel).floatValue(); + final float onsAsType = + (onlyNullsSentinel == null) ? NULL_FLOAT : ((Number) onlyNullsSentinel).floatValue(); + final float nusAsType = + (nonUniqueSentinel == null) ? NULL_FLOAT : ((Number) nonUniqueSentinel).floatValue(); return isRollup ? new FloatRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) : new FloatChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Integer.class || type == int.class) { @@ -167,12 +171,15 @@ static IterativeChunkedAggregationOperator createUnique(Class type, String re return isRollup ? new LongRollupUniqueOperator(type, resultName, countNulls, onsAsType, nusAsType) : new LongChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else if (type == Short.class || type == short.class) { - final short onsAsType = (onlyNullsSentinel == null) ? NULL_SHORT : ((Number) onlyNullsSentinel).shortValue(); - final short nusAsType = (nonUniqueSentinel == null) ? NULL_SHORT : ((Number) nonUniqueSentinel).shortValue(); + final short onsAsType = + (onlyNullsSentinel == null) ? NULL_SHORT : ((Number) onlyNullsSentinel).shortValue(); + final short nusAsType = + (nonUniqueSentinel == null) ? NULL_SHORT : ((Number) nonUniqueSentinel).shortValue(); return isRollup ? new ShortRollupUniqueOperator(resultName, countNulls, onsAsType, nusAsType) : new ShortChunkedUniqueOperator(resultName, countNulls, exposeInternal, onsAsType, nusAsType); } else { - return isRollup ? new ObjectRollupUniqueOperator(type, resultName, countNulls, onlyNullsSentinel, nonUniqueSentinel) + return isRollup + ? new ObjectRollupUniqueOperator(type, resultName, countNulls, onlyNullsSentinel, nonUniqueSentinel) : new ObjectChunkedUniqueOperator(type, resultName, countNulls, exposeInternal, onlyNullsSentinel, nonUniqueSentinel); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java index 82394d2d9ff..7c329a25c67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongChunkedDistinctOperator.java @@ -9,7 +9,7 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct; -import io.deephaven.engine.table.impl.by.RollupConstants; +import io.deephaven.engine.table.impl.sources.BoxedColumnSource; import io.deephaven.time.DateTime; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java index 8863531717d..6a523a3c0ff 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/distinct/LongRollupDistinctOperator.java @@ -9,7 +9,7 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.distinct; -import io.deephaven.engine.table.impl.by.RollupConstants; +import io.deephaven.engine.table.impl.sources.BoxedColumnSource; import io.deephaven.time.DateTime; import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; @@ -17,6 +17,7 @@ import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java index 81d463402ce..88f3fd15f92 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongChunkedUniqueOperator.java @@ -9,14 +9,15 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.unique; -import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.table.impl.sources.BoxedColumnSource; import io.deephaven.time.DateTime; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java index 381675610de..6dd9db6606b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmcountdistinct/unique/LongRollupUniqueOperator.java @@ -9,14 +9,15 @@ package io.deephaven.engine.table.impl.by.ssmcountdistinct.unique; -import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.table.impl.sources.BoxedColumnSource; import io.deephaven.time.DateTime; +import io.deephaven.engine.table.impl.by.ssmcountdistinct.DateTimeSsmSourceWrapper; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.table.TableUpdate; +import io.deephaven.engine.table.impl.by.RollupConstants; import io.deephaven.engine.updategraph.UpdateCommitter; import io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.BucketSsmDistinctRollupContext; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java index 8dce82a4f82..5c569b84687 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractRangeFilter.java @@ -22,8 +22,7 @@ public abstract class AbstractRangeFilter extends WhereFilterImpl { private static final Pattern decimalPattern = Pattern.compile("(-)?\\d+(?:\\.((\\d+)0*)?)?"); - protected final String - columnName; + protected final String columnName; protected final boolean upperInclusive; protected final boolean lowerInclusive; diff --git a/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java b/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java index 9862d4239ed..08feed7fcce 100644 --- a/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java +++ b/engine/table/src/main/java/io/deephaven/engine/util/TotalsTableBuilder.java @@ -61,8 +61,8 @@ public enum AggType { /** Collect the distinct items from the column */ Distinct, /** - * Display the singular value from the column if it is unique, or a default value if none are present, or it is not - * unique + * Display the singular value from the column if it is unique, or a default value if none are present, or it is + * not unique */ Unique, /** Only valid in a TotalsTableBuilder to indicate we should not perform any aggregation. */ diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 2db57737ccc..ba41b9ee61c 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -2481,8 +2481,7 @@ public void testTDigestMulti() { final Collection aggregations = List.of( AggApproxPct("doubleCol", PctOut(0.75, "DP75"), PctOut(0.95, "DP95"), PctOut(0.99, "DP99"), PctOut(0.999, "DP999")), - AggApproxPct("floatCol", PctOut(0.75, "FP75"), PctOut(0.99, "FP99")) - ); + AggApproxPct("floatCol", PctOut(0.75, "FP75"), PctOut(0.99, "FP99"))); final Table aggregated = queryTable.dropColumns("Sym").aggBy(aggregations); TableTools.showWithRowSet(aggregated); @@ -2509,15 +2508,13 @@ public void testTDigestAccumulation() { final Collection aggregations33 = List.of( AggTDigest(33, "Digest=doubleCol"), - AggApproxPct("doubleCol", PctOut(0.95, "P95")) - ); + AggApproxPct("doubleCol", PctOut(0.95, "P95"))); final Table aggregated = queryTable.dropColumns("Sym").aggBy(aggregations33); TableTools.showWithRowSet(aggregated); final Collection aggregations100 = List.of( AggTDigest(100, "Digest=doubleCol"), - AggApproxPct("doubleCol", PctOut(0.95, "P95")) - ); + AggApproxPct("doubleCol", PctOut(0.95, "P95"))); final Table aggregatedBySym = queryTable.aggBy(aggregations100, "Sym"); TableTools.showWithRowSet(aggregatedBySym); @@ -2617,8 +2614,7 @@ public void testTDigestIncremental() { AggApproxPct("doubleCol", PctOut(0.75, "DP75"), PctOut(0.95, "DP95"), PctOut(0.99, "DP99"), PctOut(0.999, "DP999")), AggApproxPct("longCol", PctOut(0.75, "LP75"), PctOut(0.95, "LP95"), PctOut(0.99, "LP99"), - PctOut(0.999, "LP999")) - ); + PctOut(0.999, "LP999"))); final EvalNugget[] en = new EvalNugget[] { new EvalNugget() { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java index eb80f65cfe8..9c73e57c96f 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java @@ -191,16 +191,14 @@ private void testComboByIncremental(final String ctxt, final int size) { public Table e() { return queryTable.aggBy(List.of( AggAvg("MeanI=intCol", "MeanD=doubleCol"), - AggStd("StdI=intCol", "StdD=doubleCol") - ), "Sym").sort("Sym"); + AggStd("StdI=intCol", "StdD=doubleCol")), "Sym").sort("Sym"); } }, new EvalNugget() { public Table e() { return queryTable.aggBy(List.of( AggFormula("min(each)", "each", "MinI=intCol", "MinD=doubleCol"), - AggFormula("max(each)", "each", "MaxI=intCol") - ), "Sym").sort("Sym"); + AggFormula("max(each)", "each", "MaxI=intCol")), "Sym").sort("Sym"); } }, new QueryTableTest.TableComparator( @@ -343,7 +341,7 @@ public Table e() { AggDistinct(true, "dicN=intColNulls", "didN=doubleColNulls"), AggUnique("uic=intCol", "uid=doubleCol"), AggUnique(true, "uicN=intColNulls", "uidN=doubleColNulls")), "Sym") - .sort("Sym"), + .sort("Sym"), "AggCountDistinct") }; final int steps = 100; // 8; @@ -670,7 +668,7 @@ private void testUniqueOutOfRangeParams(Class type, Table dataTable, Number i } private static void expectException(@SuppressWarnings("SameParameterValue") Class excType, - String failMessage, Runnable action) { + String failMessage, Runnable action) { try { action.run(); fail(failMessage); diff --git a/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java b/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java index bdab79b79de..dd3809451e6 100644 --- a/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java +++ b/server/src/main/java/io/deephaven/server/table/ops/ComboAggregateGrpcImpl.java @@ -163,21 +163,21 @@ private static Table comboAggregateHelper(final Table parent, final SelectColumn agg -> getColumnPairs(parent, groupByColumnSet, agg); final Collection aggregations = aggregates.stream().map( - agg -> makeAggregation(agg, getPairs) - ).collect(Collectors.toList()); + agg -> makeAggregation(agg, getPairs)).collect(Collectors.toList()); return parent.aggBy(aggregations, Arrays.asList(groupByColumns)); } private static String[] getColumnPairs(@NotNull final Table parent, - @NotNull final Set groupByColumnSet, - @NotNull final ComboAggregateRequest.Aggregate agg) { + @NotNull final Set groupByColumnSet, + @NotNull final ComboAggregateRequest.Aggregate agg) { if (agg.getMatchPairsCount() == 0) { // If not specified, we apply the aggregate to all columns not "otherwise involved" return parent.getDefinition().getColumnStream() .map(ColumnDefinition::getName) .filter(n -> !(groupByColumnSet.contains(n) || - (agg.getType() == ComboAggregateRequest.AggType.WEIGHTED_AVG && agg.getColumnName().equals(n)))) + (agg.getType() == ComboAggregateRequest.AggType.WEIGHTED_AVG + && agg.getColumnName().equals(n)))) .toArray(String[]::new); } return agg.getMatchPairsList().toArray(String[]::new); From d26a43eb6cb42246f3edd8be0cb256ffda092726 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 00:29:41 -0500 Subject: [PATCH 27/44] Fixup build issue for reflective replication --- buildSrc/src/main/groovy/Tasks.groovy | 2 ++ replication/reflective/build.gradle | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/buildSrc/src/main/groovy/Tasks.groovy b/buildSrc/src/main/groovy/Tasks.groovy index 89c851f064c..77193c21a82 100644 --- a/buildSrc/src/main/groovy/Tasks.groovy +++ b/buildSrc/src/main/groovy/Tasks.groovy @@ -18,6 +18,7 @@ class Tasks { task.workingDir project.rootDir task.classpath = sourceSet.runtimeClasspath task.main = mainClassName + task.systemProperty 'Configuration.rootFile', 'dh-defaults.prop' } } @@ -28,6 +29,7 @@ class Tasks { task.workingDir project.rootDir task.classpath = sourceSet.runtimeClasspath task.main = mainClassName + task.systemProperty 'Configuration.rootFile', 'dh-tests.prop' } } } diff --git a/replication/reflective/build.gradle b/replication/reflective/build.gradle index ecee04df9bf..d00059aee36 100644 --- a/replication/reflective/build.gradle +++ b/replication/reflective/build.gradle @@ -7,6 +7,11 @@ description 'Reflective Replicators: Source code generators and replicators with dependencies { implementation project(':replication-util'), project(':engine-table') + + runtimeOnly project(':log-to-slf4j'), + project(path: ':configs'), + project(path: ':test-configs') + Classpaths.inheritSlf4j(project, 'slf4j-simple', 'runtimeOnly') } task replicateAll { From 781b5c1ef81be6c4527f3de350726087a201b8ff Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 00:37:50 -0500 Subject: [PATCH 28/44] Delete non-operator by aggregation code --- .../table/impl/by/AggregationControl.java | 6 +- .../table/impl/by/AggregationHelper.java | 589 ------ ...IncrementalByAggregationUpdateTracker.java | 620 ------ ...entalChunkedByAggregationStateManager.java | 1697 ----------------- ...hunkedOperatorAggregationStateManager.java | 2 +- ...taticChunkedByAggregationStateManager.java | 1357 ------------- .../replicators/ReplicateHashTable.java | 11 - 7 files changed, 4 insertions(+), 4278 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationHelper.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalByAggregationUpdateTracker.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java index da7e0047147..5926354e02d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationControl.java @@ -22,15 +22,15 @@ public boolean considerGrouping(@NotNull Table table, @NotNull ColumnSource[] public int initialHashTableSize(@NotNull final Table inputTable) { // TODO: This approach relies on rehash. Maybe we should consider sampling instead. - return IncrementalChunkedByAggregationStateManager.MINIMUM_INITIAL_HASH_SIZE; + return IncrementalChunkedOperatorAggregationStateManager.MINIMUM_INITIAL_HASH_SIZE; } public double getTargetLoadFactor() { - return IncrementalChunkedByAggregationStateManager.DEFAULT_TARGET_LOAD_FACTOR; + return IncrementalChunkedOperatorAggregationStateManager.DEFAULT_TARGET_LOAD_FACTOR; } public double getMaximumLoadFactor() { - return IncrementalChunkedByAggregationStateManager.DEFAULT_MAX_LOAD_FACTOR; + return IncrementalChunkedOperatorAggregationStateManager.DEFAULT_MAX_LOAD_FACTOR; } public boolean considerGrouping(@NotNull final Table inputTable, @NotNull final ColumnSource[] sources) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationHelper.java deleted file mode 100644 index 03da6cbd321..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationHelper.java +++ /dev/null @@ -1,589 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.base.Pair; -import io.deephaven.base.verify.Assert; -import io.deephaven.datastructures.util.SmartKey; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.table.*; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.RowSequenceFactory; -import io.deephaven.engine.table.impl.GroupingUtils; -import io.deephaven.engine.table.impl.TableUpdateImpl; -import io.deephaven.engine.table.impl.*; -import io.deephaven.engine.table.impl.indexer.RowSetIndexer; -import io.deephaven.engine.table.impl.select.SelectColumn; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.sources.aggregate.AggregateColumnSource; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.chunk.*; -import io.deephaven.engine.table.impl.SmartKeySource; -import io.deephaven.engine.table.impl.util.*; -import io.deephaven.util.annotations.VisibleForTesting; -import org.apache.commons.lang3.mutable.Mutable; -import org.apache.commons.lang3.mutable.MutableObject; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; - -import java.util.*; -import java.util.stream.Collectors; - -import static io.deephaven.datastructures.util.CollectionUtil.ZERO_LENGTH_STRING_ARRAY; - -/** - * Implementation for chunk-oriented aggregation operations, including {@link Table#groupBy} and - * {@link Table#partitionBy}. - */ -public class AggregationHelper { - - /** - * Static-use only. - */ - private AggregationHelper() {} - - public static QueryTable by(@NotNull final QueryTable inputTable, - @NotNull final SelectColumn... keyColumns) { - return by(AggregationControl.DEFAULT, inputTable, keyColumns); - } - - @VisibleForTesting - public static QueryTable by(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final SelectColumn... keyColumns) { - // If we have no key columns aggregate all columns with no hashing - if (keyColumns.length == 0) { - return noKeyBy(inputTable); - } - - // Compute our key column sources - final Map> existingColumnSourceMap = inputTable.getColumnSourceMap(); - final Set keyColumnUpstreamInputColumnNames = new HashSet<>(keyColumns.length); - final String[] keyColumnNames; - final String[] aggregatedColumnNames; - final ColumnSource[] keyColumnSources; - { - final Map> keyColumnSourceMap = new LinkedHashMap<>(keyColumns.length); - final Map> fullColumnSourceMap = new LinkedHashMap<>(existingColumnSourceMap); - Arrays.stream(keyColumns).forEachOrdered((final SelectColumn keyColumn) -> { - keyColumn.initInputs(inputTable.getRowSet(), fullColumnSourceMap); - - // Accumulate our key column inputs - final Set thisKeyColumnUpstreamInputColumnNames = new HashSet<>(); - thisKeyColumnUpstreamInputColumnNames.addAll(keyColumn.getColumns()); - thisKeyColumnUpstreamInputColumnNames.addAll(keyColumn.getColumnArrays()); - thisKeyColumnUpstreamInputColumnNames.removeAll(keyColumnSourceMap.keySet()); - keyColumnUpstreamInputColumnNames.addAll(thisKeyColumnUpstreamInputColumnNames); - - // Accumulate our column source maps - final ColumnSource keyColumnSource = keyColumn.getDataView(); - fullColumnSourceMap.put(keyColumn.getName(), keyColumnSource); - keyColumnSourceMap.put(keyColumn.getName(), keyColumnSource); - }); - keyColumnNames = keyColumnSourceMap.keySet().toArray(ZERO_LENGTH_STRING_ARRAY); - aggregatedColumnNames = existingColumnSourceMap.keySet().stream() - .filter(columnSource -> !keyColumnSourceMap.containsKey(columnSource)).toArray(String[]::new); - keyColumnSources = keyColumnSourceMap.values().toArray(ColumnSource.ZERO_LENGTH_COLUMN_SOURCE_ARRAY); - } - - // If we can use an existing static grouping, convert that to a table - final Map groupingForAggregation = - maybeGetGroupingForAggregation(aggregationControl, inputTable, keyColumnSources); - if (groupingForAggregation != null) { - // noinspection unchecked - return staticGroupedBy(existingColumnSourceMap, keyColumnNames[0], - (ColumnSource) keyColumnSources[0], groupingForAggregation); - } - - // Perform a full hashtable backed aggregation - if (inputTable.isRefreshing()) { - return incrementalHashedBy(aggregationControl, inputTable, existingColumnSourceMap, keyColumnNames, - aggregatedColumnNames, keyColumnSources, keyColumnUpstreamInputColumnNames); - } - return staticHashedBy(aggregationControl, inputTable, existingColumnSourceMap, keyColumnNames, - aggregatedColumnNames, keyColumnSources); - } - - @NotNull - private static QueryTable noKeyBy(@NotNull final QueryTable inputTable) { - final Mutable resultHolder = new MutableObject<>(); - final SwapListener swapListener = - inputTable.createSwapListenerIfRefreshing(SwapListener::new); - inputTable.initializeWithSnapshot("groupBy()-Snapshot", swapListener, - (final boolean usePrev, final long beforeClockValue) -> { - final ColumnSource resultIndexColumnSource = - new SingleValueObjectColumnSource<>((TrackingWritableRowSet) inputTable.getRowSet()); - final boolean empty = - usePrev ? inputTable.getRowSet().firstRowKeyPrev() == RowSequence.NULL_ROW_KEY - : inputTable.isEmpty(); - final QueryTable resultTable = new QueryTable( - RowSetFactory.flat(empty ? 0 : 1).toTracking(), - inputTable.getColumnSourceMap().entrySet().stream().collect(Collectors.toMap( - Map.Entry::getKey, - (final Map.Entry> columnNameToSourceEntry) -> { - final AggregateColumnSource aggregateColumnSource = AggregateColumnSource - .make(columnNameToSourceEntry.getValue(), resultIndexColumnSource); - aggregateColumnSource.startTrackingPrevValues(); - return aggregateColumnSource; - }, - Assert::neverInvoked, - LinkedHashMap::new))); - if (swapListener != null) { - final ModifiedColumnSet.Transformer transformer = inputTable.newModifiedColumnSetTransformer( - inputTable.getDefinition().getColumnNamesArray(), - resultTable.getDefinition().getColumnNames().stream() - .map(resultTable::newModifiedColumnSet).toArray(ModifiedColumnSet[]::new)); - final TableUpdateListener aggregationUpdateListener = - new BaseTable.ListenerImpl("groupBy()", inputTable, resultTable) { - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - final boolean wasEmpty = - inputTable.getRowSet().firstRowKeyPrev() == RowSequence.NULL_ROW_KEY; - final boolean isEmpty = inputTable.getRowSet().isEmpty(); - final RowSet added; - final RowSet removed; - final RowSet modified; - final ModifiedColumnSet modifiedColumnSet; - if (wasEmpty) { - if (isEmpty) { - // empty -> empty: No change to report, we probably shouldn't even be - // notified - return; - } - resultTable.getRowSet().writableCast().insert(0); - added = RowSetFactory.flat(1); - removed = RowSetFactory.empty(); - modified = RowSetFactory.empty(); - modifiedColumnSet = ModifiedColumnSet.EMPTY; - } else if (isEmpty) { - resultTable.getRowSet().writableCast().remove(0); - added = RowSetFactory.empty(); - removed = RowSetFactory.flat(1); - modified = RowSetFactory.empty(); - modifiedColumnSet = ModifiedColumnSet.EMPTY; - } else if (upstream.added().isNonempty() || upstream.removed().isNonempty()) { - added = RowSetFactory.empty(); - removed = RowSetFactory.empty(); - modified = RowSetFactory.flat(1); - modifiedColumnSet = ModifiedColumnSet.ALL; - } else if (upstream.modified().isNonempty()) { - added = RowSetFactory.empty(); - removed = RowSetFactory.empty(); - modified = RowSetFactory.flat(1); - transformer.clearAndTransform(upstream.modifiedColumnSet(), - modifiedColumnSet = resultTable.getModifiedColumnSetForUpdates()); - } else { - // Only shifts: Nothing to report downstream, our data has not changed - return; - } - final TableUpdate downstream = new TableUpdateImpl(added, removed, modified, - RowSetShiftData.EMPTY, modifiedColumnSet); - resultTable.notifyListeners(downstream); - } - }; - swapListener.setListenerAndResult(aggregationUpdateListener, resultTable); - resultTable.addParentReference(swapListener); - } - resultHolder.setValue(resultTable); - return true; - }); - - - return resultHolder.getValue(); - } - - @NotNull - private static QueryTable staticGroupedBy(@NotNull final Map> existingColumnSourceMap, - @NotNull final String keyColumnName, - @NotNull final ColumnSource keyColumnSource, - @NotNull final Map groupToIndex) { - final Pair, ObjectArraySource> flatResultColumnSources = - GroupingUtils.groupingToFlatSources(keyColumnSource, groupToIndex); - final ArrayBackedColumnSource resultKeyColumnSource = flatResultColumnSources.getFirst(); - final ObjectArraySource resultIndexColumnSource = flatResultColumnSources.getSecond(); - - final TrackingWritableRowSet resultRowSet = RowSetFactory.flat(groupToIndex.size()).toTracking(); - final Map> resultColumnSourceMap = new LinkedHashMap<>(); - resultColumnSourceMap.put(keyColumnName, resultKeyColumnSource); - existingColumnSourceMap.entrySet().stream() - .filter((final Map.Entry> columnNameToSourceEntry) -> !columnNameToSourceEntry - .getKey().equals(keyColumnName)) - .forEachOrdered( - (final Map.Entry> columnNameToSourceEntry) -> resultColumnSourceMap - .put(columnNameToSourceEntry.getKey(), AggregateColumnSource - .make(columnNameToSourceEntry.getValue(), resultIndexColumnSource))); - - return new QueryTable(resultRowSet, resultColumnSourceMap); - } - - @NotNull - private static QueryTable staticHashedBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final Map> existingColumnSourceMap, - @NotNull final String[] keyColumnNames, - @NotNull final String[] aggregatedColumnNames, - @NotNull final ColumnSource[] keyColumnSources) { - // Reinterpret key column sources as primitives where possible - final ColumnSource[] maybeReinterpretedKeyColumnSources = maybeReinterpretKeyColumnSources(keyColumnSources); - - // Prepare our state manager - final StaticChunkedByAggregationStateManager stateManager = - new StaticChunkedByAggregationStateManager(maybeReinterpretedKeyColumnSources, - aggregationControl.initialHashTableSize(inputTable), aggregationControl.getTargetLoadFactor(), - aggregationControl.getMaximumLoadFactor()); - - // Do the actual aggregation hashing and convert the results - final IntegerArraySource groupIndexToHashSlot = new IntegerArraySource(); - final int numGroups = - stateManager.buildTable(inputTable, maybeReinterpretedKeyColumnSources, groupIndexToHashSlot); - stateManager.convertBuildersToIndexes(groupIndexToHashSlot, numGroups); - - // TODO: Consider selecting the hash inputTable sources, in order to truncate them to size and improve density - - // Compute result RowSet and redirection to hash slots - final TrackingRowSet resultRowSet = RowSetFactory.flat(numGroups).toTracking(); - final WritableRowRedirection resultIndexToHashSlot = - new IntColumnSourceWritableRowRedirection(groupIndexToHashSlot); - - // Construct result column sources - final ColumnSource[] keyHashTableSources = stateManager.getKeyHashTableSources(); - final Map> resultColumnSourceMap = new LinkedHashMap<>(); - - // Gather the result key columns - for (int kci = 0; kci < keyHashTableSources.length; ++kci) { - ColumnSource resultKeyColumnSource = keyHashTableSources[kci]; - if (keyColumnSources[kci] != maybeReinterpretedKeyColumnSources[kci]) { - resultKeyColumnSource = - ReinterpretUtils.convertToOriginal(keyColumnSources[kci].getType(), resultKeyColumnSource); - } - resultColumnSourceMap.put(keyColumnNames[kci], - new RedirectedColumnSource<>(resultIndexToHashSlot, resultKeyColumnSource)); - } - - // Gather the result aggregate columns - final ColumnSource resultIndexColumnSource = - new RedirectedColumnSource<>(resultIndexToHashSlot, stateManager.getIndexHashTableSource()); - Arrays.stream(aggregatedColumnNames) - .forEachOrdered((final String aggregatedColumnName) -> resultColumnSourceMap.put(aggregatedColumnName, - AggregateColumnSource.make(existingColumnSourceMap.get(aggregatedColumnName), - resultIndexColumnSource))); - - // Construct the result table - return new QueryTable(resultRowSet, resultColumnSourceMap); - } - - @NotNull - private static QueryTable incrementalHashedBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final Map> existingColumnSourceMap, - @NotNull final String[] keyColumnNames, - @NotNull final String[] aggregatedColumnNames, - @NotNull final ColumnSource[] keyColumnSources, - @NotNull final Set keyColumnUpstreamInputColumnNames) { - final Mutable resultHolder = new MutableObject<>(); - final SwapListener swapListener = - inputTable.createSwapListenerIfRefreshing(SwapListener::new); - assert swapListener != null; - inputTable.initializeWithSnapshot("groupBy(" + String.join(",", keyColumnNames) + "-Snapshot", swapListener, - (final boolean usePrev, final long beforeClockValue) -> { - // Reinterpret key column sources as primitives where possible - final ColumnSource[] maybeReinterpretedKeyColumnSources = - maybeReinterpretKeyColumnSources(keyColumnSources); - - // Prepare our state manager - final IncrementalChunkedByAggregationStateManager stateManager = - new IncrementalChunkedByAggregationStateManager(maybeReinterpretedKeyColumnSources, - aggregationControl.initialHashTableSize(inputTable), - aggregationControl.getTargetLoadFactor(), - aggregationControl.getMaximumLoadFactor()); - - // Prepare our update tracker - final IncrementalByAggregationUpdateTracker updateTracker = - new IncrementalByAggregationUpdateTracker(); - - // Perform the initial aggregation pass - if (usePrev) { - stateManager.buildInitialTableFromPrevious(inputTable, maybeReinterpretedKeyColumnSources, - updateTracker); - } else { - stateManager.buildInitialTableFromCurrent(inputTable, maybeReinterpretedKeyColumnSources, - updateTracker); - - } - // Compute result RowSet and redirection to hash slots - final WritableRowRedirection resultIndexToHashSlot = - WritableRowRedirectionLockFree.FACTORY.createRowRedirection(updateTracker.size()); - final TrackingWritableRowSet resultRowSet = updateTracker.applyAddsAndMakeInitialRowSet( - stateManager.getRowSetSource(), stateManager.getOverflowRowSetSource(), - resultIndexToHashSlot); - - // Construct result column sources - final Map> resultColumnSourceMap = new LinkedHashMap<>(); - - // Gather the result key columns - for (int kci = 0; kci < keyColumnSources.length; ++kci) { - resultColumnSourceMap.put(keyColumnNames[kci], keyColumnSources[kci]); - } - - // Gather the result aggregate columns - final ColumnSource resultIndexColumnSource = - new RedirectedColumnSource<>( - resultIndexToHashSlot, stateManager.getRowSetHashTableSource()); - Arrays.stream(aggregatedColumnNames) - .forEachOrdered((final String aggregatedColumnName) -> { - final AggregateColumnSource aggregatedColumnSource = AggregateColumnSource.make( - existingColumnSourceMap.get(aggregatedColumnName), resultIndexColumnSource); - aggregatedColumnSource.startTrackingPrevValues(); - resultColumnSourceMap.put(aggregatedColumnName, aggregatedColumnSource); - }); - - // Construct the result table - final QueryTable resultTable = new QueryTable(resultRowSet, resultColumnSourceMap); - resultIndexToHashSlot.startTrackingPrevValues(); - - // Categorize modified column sets - final ModifiedColumnSet upstreamKeyColumnInputs = inputTable - .newModifiedColumnSet(keyColumnUpstreamInputColumnNames.toArray(ZERO_LENGTH_STRING_ARRAY)); - final ModifiedColumnSet downstreamAllAggregatedColumns = - resultTable.newModifiedColumnSet(aggregatedColumnNames); - final ModifiedColumnSet.Transformer aggregatedColumnsTransformer = - inputTable.newModifiedColumnSetTransformer( - aggregatedColumnNames, - Arrays.stream(aggregatedColumnNames).map(resultTable::newModifiedColumnSet) - .toArray(ModifiedColumnSet[]::new)); - - // Handle updates - final TableUpdateListener aggregationUpdateListener = new BaseTable.ListenerImpl( - "groupBy(" + String.join(",", keyColumnNames) + ')', inputTable, resultTable) { - @Override - public void onUpdate(@NotNull final TableUpdate upstream) { - if (updateTracker.clear()) { - stateManager.clearCookies(); - } - - final boolean keyColumnsModified = - upstream.modifiedColumnSet().containsAny(upstreamKeyColumnInputs); - - if (keyColumnsModified) { - try (final RowSet toRemove = upstream.removed().union(upstream.getModifiedPreShift())) { - stateManager.processRemoves(maybeReinterpretedKeyColumnSources, toRemove, - updateTracker); - } - } else { - stateManager.processRemoves(maybeReinterpretedKeyColumnSources, upstream.removed(), - updateTracker); - } - updateTracker.applyRemovesToStates(stateManager.getRowSetSource(), - stateManager.getOverflowRowSetSource()); - - if (upstream.shifted().nonempty()) { - upstream.shifted() - .apply((final long beginRange, final long endRange, final long shiftDelta) -> { - final RowSet shiftedPreviousRowSet; - try (final RowSet previousIndex = inputTable.getRowSet().copyPrev()) { - shiftedPreviousRowSet = - previousIndex.subSetByKeyRange(beginRange, endRange); - } - try { - if (aggregationControl.shouldProbeShift(shiftedPreviousRowSet.size(), - resultRowSet.intSize())) { - stateManager.processShift(maybeReinterpretedKeyColumnSources, - shiftedPreviousRowSet, updateTracker); - updateTracker.applyShiftToStates(stateManager.getRowSetSource(), - stateManager.getOverflowRowSetSource(), beginRange, - endRange, - shiftDelta); - } else { - resultRowSet.forAllRowKeys((final long stateKey) -> { - final int stateSlot = (int) resultIndexToHashSlot.get(stateKey); - stateManager.applyShift(stateSlot, beginRange, endRange, - shiftDelta, updateTracker::processAppliedShift); - }); - } - } finally { - shiftedPreviousRowSet.close(); - } - }); - } - - if (keyColumnsModified) { - try (final RowSet toAdd = upstream.added().union(upstream.modified())) { - stateManager.processAdds(maybeReinterpretedKeyColumnSources, toAdd, updateTracker); - } - } else { - stateManager.processModifies(maybeReinterpretedKeyColumnSources, upstream.modified(), - updateTracker); - stateManager.processAdds(maybeReinterpretedKeyColumnSources, upstream.added(), - updateTracker); - } - updateTracker.applyAddsToStates(stateManager.getRowSetSource(), - stateManager.getOverflowRowSetSource()); - - final TableUpdate downstream = updateTracker.makeUpdateFromStates( - stateManager.getRowSetSource(), stateManager.getOverflowRowSetSource(), - resultRowSet, - resultIndexToHashSlot, - (final boolean someKeyHasAddsOrRemoves, final boolean someKeyHasModifies) -> { - if (someKeyHasAddsOrRemoves) { - return downstreamAllAggregatedColumns; - } - if (someKeyHasModifies) { - aggregatedColumnsTransformer.clearAndTransform(upstream.modifiedColumnSet(), - resultTable.getModifiedColumnSetForUpdates()); - return resultTable.getModifiedColumnSetForUpdates(); - } - return ModifiedColumnSet.EMPTY; - }); - resultTable.notifyListeners(downstream); - } - }; - swapListener.setListenerAndResult(aggregationUpdateListener, resultTable); - resultTable.addParentReference(swapListener); - - resultHolder.setValue(resultTable); - return true; - }); - - return resultHolder.getValue(); - } - - public static LocalTableMap partitionBy(@NotNull final QueryTable inputTable, - final boolean dropKeyColumns, - @NotNull final String... keyColumnNames) { - return partitionBy(AggregationControl.DEFAULT, inputTable, dropKeyColumns, keyColumnNames); - } - - @VisibleForTesting - public static LocalTableMap partitionBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - final boolean dropKeyColumns, - @NotNull final String... keyColumnNames) { - // If there are no key columns, return a map with just the input table; there's nothing to be aggregated - if (keyColumnNames.length == 0) { - final LocalTableMap noKeyResult = new LocalTableMap(null, inputTable.getDefinition()); - noKeyResult.put(SmartKey.EMPTY, inputTable); - return noKeyResult; - } - - final ColumnSource[] keyColumnSources = - Arrays.stream(keyColumnNames).map(inputTable::getColumnSource).toArray(ColumnSource[]::new); - final QueryTable subTableSource = - dropKeyColumns ? (QueryTable) inputTable.dropColumns(keyColumnNames) : inputTable; - - // If we can use an existing static grouping, trivially convert that to a table map - final Map groupingForAggregation = - maybeGetGroupingForAggregation(aggregationControl, inputTable, keyColumnSources); - if (groupingForAggregation != null) { - final LocalTableMap staticGroupedResult = new LocalTableMap(null, inputTable.getDefinition()); - GroupingUtils.forEachResponsiveGroup(groupingForAggregation, inputTable.getRowSet(), - (final Object key, final WritableRowSet rowSet) -> staticGroupedResult.put(key, - subTableSource.getSubTable(rowSet.toTracking()))); - return staticGroupedResult; - } - - if (inputTable.isRefreshing()) { - return incrementalHashedPartitionBy(aggregationControl, inputTable, keyColumnSources, subTableSource); - } - - return staticHashedPartitionBy(aggregationControl, inputTable, keyColumnSources, subTableSource); - } - - @NotNull - private static LocalTableMap staticHashedPartitionBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final ColumnSource[] keyColumnSources, - @NotNull final QueryTable subTableSource) { - // Reinterpret key column sources as primitives where possible - final ColumnSource[] maybeReinterpretedKeyColumnSources = maybeReinterpretKeyColumnSources(keyColumnSources); - - // Prepare our state manager - final StaticChunkedByAggregationStateManager stateManager = - new StaticChunkedByAggregationStateManager(maybeReinterpretedKeyColumnSources, - aggregationControl.initialHashTableSize(inputTable), aggregationControl.getTargetLoadFactor(), - aggregationControl.getMaximumLoadFactor()); - - // Do the actual aggregation hashing and convert the results - final IntegerArraySource groupIndexToHashSlot = new IntegerArraySource(); - final int numGroups = - stateManager.buildTable(inputTable, maybeReinterpretedKeyColumnSources, groupIndexToHashSlot); - stateManager.convertBuildersToIndexes(groupIndexToHashSlot, numGroups); - - // Build our table map - final LocalTableMap staticHashedResult = new LocalTableMap(null, inputTable.getDefinition()); - - final TupleSource inputKeyIndexToMapKeySource = - keyColumnSources.length == 1 ? keyColumnSources[0] : new SmartKeySource(keyColumnSources); - final ColumnSource hashSlotToRowSetSource = stateManager.getIndexHashTableSource(); - final int chunkSize = Math.min(numGroups, IncrementalChunkedByAggregationStateManager.CHUNK_SIZE); - - try (final RowSequence groupIndices = RowSetFactory.flat(numGroups); - final RowSequence.Iterator groupIndicesIterator = groupIndices.getRowSequenceIterator(); - final ChunkSource.GetContext hashSlotGetContext = groupIndexToHashSlot.makeGetContext(chunkSize); - final WritableObjectChunk aggregatedIndexes = - WritableObjectChunk.makeWritableChunk(chunkSize); - final WritableLongChunk mapKeySourceIndices = - WritableLongChunk.makeWritableChunk(chunkSize); - final ChunkSource.GetContext mapKeyGetContext = inputKeyIndexToMapKeySource.makeGetContext(chunkSize)) { - while (groupIndicesIterator.hasMore()) { - final RowSequence groupIndexesForThisChunk = - groupIndicesIterator.getNextRowSequenceWithLength(chunkSize); - final int groupsInThisChunk = groupIndexesForThisChunk.intSize(); - final LongChunk hashSlots = - groupIndexToHashSlot.getChunk(hashSlotGetContext, groupIndexesForThisChunk).asLongChunk(); - for (int gi = 0; gi < groupsInThisChunk; ++gi) { - final TrackingWritableRowSet rowSet = hashSlotToRowSetSource.get(hashSlots.get(gi)); - aggregatedIndexes.set(gi, rowSet); - mapKeySourceIndices.set(gi, rowSet.firstRowKey()); - } - aggregatedIndexes.setSize(groupsInThisChunk); - mapKeySourceIndices.setSize(groupsInThisChunk); - final ObjectChunk mapKeys; - try (final RowSequence inputKeyIndices = - RowSequenceFactory.wrapRowKeysChunkAsRowSequence(mapKeySourceIndices)) { - mapKeys = inputKeyIndexToMapKeySource.getChunk(mapKeyGetContext, inputKeyIndices).asObjectChunk(); - } - for (int gi = 0; gi < groupsInThisChunk; ++gi) { - staticHashedResult.put(mapKeys.get(gi), subTableSource.getSubTable(aggregatedIndexes.get(gi))); - } - } - } - return staticHashedResult; - } - - @NotNull - private static LocalTableMap incrementalHashedPartitionBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final ColumnSource[] keyColumnSources, - @NotNull final QueryTable subTableSource) { - throw new UnsupportedOperationException("Never developed"); - } - - @Nullable - private static Map maybeGetGroupingForAggregation( - @NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final ColumnSource[] keyColumnSources) { - // If we have one grouped key column and the input table is not refreshing use the existing grouping - if (!aggregationControl.considerGrouping(inputTable, keyColumnSources)) { - return null; - } - // noinspection unchecked - final ColumnSource keyColumnSource = (ColumnSource) keyColumnSources[0]; - final RowSetIndexer indexer = RowSetIndexer.of(inputTable.getRowSet()); - if (indexer.hasGrouping(keyColumnSource)) { - return indexer.getGrouping(keyColumnSource); - } - return null; - } - - @NotNull - private static ColumnSource[] maybeReinterpretKeyColumnSources( - @NotNull final ColumnSource[] keyColumnSources) { - // TODO: Support symbol tables in reinterpret and re-boxing - final ColumnSource[] maybeReinterpretedKeyColumnSources = new ColumnSource[keyColumnSources.length]; - for (int kci = 0; kci < keyColumnSources.length; ++kci) { - maybeReinterpretedKeyColumnSources[kci] = - ReinterpretUtils.maybeConvertToPrimitive(keyColumnSources[kci]); - } - return maybeReinterpretedKeyColumnSources; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalByAggregationUpdateTracker.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalByAggregationUpdateTracker.java deleted file mode 100644 index 0697a46a45e..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalByAggregationUpdateTracker.java +++ /dev/null @@ -1,620 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.base.verify.Assert; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.rowset.RowSetFactory; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.TableUpdate; -import io.deephaven.engine.table.impl.TableUpdateImpl; -import io.deephaven.engine.table.ModifiedColumnSet; -import io.deephaven.engine.table.impl.sort.timsort.LongLongTimsortKernel; -import io.deephaven.engine.table.impl.sources.IntegerArraySource; -import io.deephaven.engine.table.impl.sources.LongArraySource; -import io.deephaven.engine.table.impl.sources.ObjectArraySource; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.engine.table.impl.util.*; -import org.jetbrains.annotations.NotNull; - -/** - *

- * A tracker for accumulating changes to aggregation states for {@link Table#groupBy}. - * - *

- * The tracker is used in the initial (insert only) build phase, as well as in subsequent update passes. - * - *

- * Update processing is performed as follows (note that flags are accumulated across steps 1-4 and used in - * step 5): - *

    - *
  1. Probe and accumulate removes (including modified-pre-shift when key columns are modified) in sequential builders - * per state, then build the removed {@link TrackingWritableRowSet} for each state and remove it from the state's - * {@link TrackingWritableRowSet}
  2. - *
  3. Probe shifts and apply them as they are found to impact a given state's {@link TrackingWritableRowSet}, writing - * down the total number of states with shifts as the chunk size for accumulating shifts in step 5
  4. - *
  5. Probe non-key modifies and flag impacted states
  6. - *
  7. Build and accumulate adds (including modified-post-shift when key columns are modified) in sequential builders - * per state, then build the added {@link TrackingWritableRowSet} for each state and add it to the state's - * {@link TrackingWritableRowSet}
  8. - *
  9. Update redirections from the previous {@link TrackingWritableRowSet} first key to the current - * {@link TrackingWritableRowSet} first key, and from old slot to new slot where a state was moved or promoted in - * rehash, accumulating row keys in 3 random builders (for added, removed, and modified) and shifts in a pair of - * parallel {@link WritableLongChunk}s for previous and current, using the following logic: - *
      - *
    1. Non-empty to empty transitions as removes of the previous first key
    2. - *
    3. Empty or null placeholder to non-empty transitions as adds of the current first key
    4. - *
    5. Shifted-only states as shifts from previous first key to current first key, appended to the paired shift - * chunks
    6. - *
    7. All other changes as modifies if first key is unchanged, else paired removes and adds if first key changed
    8. - *
    - *
  10. - *
  11. Sort the shift chunks by the previous keys, accumulate shifts into a {@link RowSetShiftData.Builder}
  12. - *
- * - *

- * In each phase, the initial addition of a state to the tracker will return a cookie, which must be passed to - * subsequent updates to the tracker for that state. - * - *

- * To process results after steps 1, 4, and 5, the caller uses - * {@link #applyRemovesToStates(ObjectArraySource, ObjectArraySource)}, - * {@link #applyAddsToStates(ObjectArraySource, ObjectArraySource)}, and - * {@link #makeUpdateFromStates(ObjectArraySource, ObjectArraySource, WritableRowSet, WritableRowRedirection, ModifiedColumnSetProducer)} - * respectively. - */ -class IncrementalByAggregationUpdateTracker { - - static final long NULL_COOKIE = 0; - - private static final long MINIMUM_COOKIE = 1; - - private static final int ALLOCATION_UNIT = 4096; - - /** - * For each updated state, store the slot its in (regardless of whether main or overflow) in the higher 7 bytes, and - * flags in the lower 1 byte. Note that flags only use 5 bits currently, but it seems reasonable to reserve a whole - * byte. - */ - private final LongArraySource updatedStateSlotAndFlags = new LongArraySource(); - - /** - * Builders (used in remove processing and add processing), parallel to {@code updatedStateSlotAndFlags}. - */ - private final ObjectArraySource builders = - new ObjectArraySource<>(RowSetBuilderSequential.class); - - /** - * Each time we clear, we add an offset to our cookies, this prevents us from reading old values. - */ - private long cookieGeneration = MINIMUM_COOKIE; - - /** - * The number of updated states, which is also the next position we will use in {@code updateStateSlotAndFlags} and - * {@code builders}. Note that cookies with implied pointers outside of {@code [0, size)} are known to be invalid. - */ - private int size; - - /** - * The number of tracker positions allocated. - */ - private int capacity; - - /** - *

- * The set of positions in {@link #updatedStateSlotAndFlags} (and possibly {@link #builders}) that have been updated - * in the current pass. Each corresponding "slot and flags" value will have the {@link #FLAG_STATE_IN_CURRENT_PASS} - * bit set. - *

- * Note that current pass membership is recorded by {@link #processShift(long, int, long)} and - * {@link #processAdd(long, int, long)}, only, and cleared in the following - * {@link #applyAddsToStates(ObjectArraySource, ObjectArraySource)} or - * {@link #applyShiftToStates(ObjectArraySource, ObjectArraySource, long, long, long)}. - */ - private final IntegerArraySource currentPassPositions = new IntegerArraySource(); - - /** - * The number of states whose "slot and flags" position can be found in in {@link #currentPassPositions}. - */ - private int currentPassSize; - - /** - * The number of "current pass" positions allocated. - */ - private int currentPassCapacity; - - // @formatter:off - private static final int FLAG_SHIFT = 8; - private static final int FLAG_MASK = 0b11111111; - private static final byte FLAG_STATE_IN_CURRENT_PASS = 0b00000001; - private static final byte FLAG_STATE_HAS_REMOVES = 0b00000010; - private static final byte FLAG_STATE_HAS_SHIFTS = 0b00000100; - private static final byte FLAG_STATE_HAS_MODIFIES = 0b00001000; - private static final byte FLAG_STATE_HAS_ADDS = 0b00010000; - // @formatter:on - - /** - * Remove all states from the tracker. - * - * @return Whether all externally-stored cookies should be reset to {@link #NULL_COOKIE} - */ - boolean clear() { - boolean needToResetCookies = false; - cookieGeneration += size; - if (cookieGeneration > Long.MAX_VALUE / 2) { - cookieGeneration = MINIMUM_COOKIE; - needToResetCookies = true; - } - size = 0; - return needToResetCookies; - } - - /** - * Get the size of this tracker, meaning the number of states with recorded updates. - * - * @return The size of the tracker - */ - int size() { - return size; - } - - /** - * Is this cookie within our valid range (greater than or equal to our generation, but less than the size after - * adjustment)? - * - * @param cookie The cookie to check for validity - * @return true if the cookie is from the current generation,and references a valid tracker position - */ - private boolean isValidCookie(final long cookie) { - return cookie >= cookieGeneration && cookieToPosition(cookie) < size; - } - - /** - * Given a position value, get a cookie for the state to store. - * - * @param position the position to convert to a cookie - * @return the cookie to return to the user - */ - private long positionToCookie(final int position) { - return cookieGeneration + position; - } - - /** - * Given a state's valid cookie, get the corresponding position. - * - * @param cookie Ghe valid cookie - * @return Ghe position in the tracker - */ - private int cookieToPosition(long cookie) { - return (int) (cookie - cookieGeneration); - } - - /** - * Record that a row key has been added to a state on initial build, to be applied in - * {@link #applyAddsAndMakeInitialRowSet(ObjectArraySource, ObjectArraySource, WritableRowRedirection)} - * - * @param cookie The last known cookie for the state - * @param stateSlot The state's slot (in main table space) - * @param addedIndex The row key that was added - * @return The new cookie for the state if it has changed - */ - final long processInitialAdd(final long cookie, final int stateSlot, final long addedIndex) { - return setFlagsAndBuild(cookie, stateSlot, FLAG_STATE_HAS_ADDS, addedIndex); - } - - /** - * Record that a row key has been removed from a state, to be applied in - * {@link #applyRemovesToStates(ObjectArraySource, ObjectArraySource)}. - * - * @param cookie The last known cookie for the state - * @param stateSlot The state's slot (in main table space) - * @param removedIndex The row key that was removed - * @return The new cookie for the state if it has changed - */ - final long processRemove(final long cookie, final int stateSlot, final long removedIndex) { - return setFlagsAndBuild(cookie, stateSlot, FLAG_STATE_HAS_REMOVES, removedIndex); - } - - /** - * Record that a row key has been shifted in a state, to be applied in - * {@link #applyShiftToStates(ObjectArraySource, ObjectArraySource, long, long, long)}. - * - * @param cookie The last known cookie for the state - * @param stateSlot The state's slot (in main table space) - * @param unusedShiftedIndex Unused shifted row key argument, so we can use a method reference with the right - * signature - * @return The new cookie for the state if it has changed - */ - final long processShift(final long cookie, final int stateSlot, - @SuppressWarnings("unused") final long unusedShiftedIndex) { - return setFlags(cookie, stateSlot, (byte) (FLAG_STATE_HAS_SHIFTS | FLAG_STATE_IN_CURRENT_PASS)); - } - - /** - * Record that a row key has been shifted in a state, already applied. - * - * @param cookie The last known cookie for the state - * @param stateSlot The state's slot (in main table space) - * @return The new cookie for the state if it has changed - */ - final long processAppliedShift(final long cookie, final int stateSlot) { - return setFlags(cookie, stateSlot, FLAG_STATE_HAS_SHIFTS); - } - - /** - * Record that a row key has been modified in a state. - * - * @param cookie The last known cookie for the state - * @param stateSlot The state's slot (in main table space) - * @param unusedModifiedIndex Unused modified row key argument, so we can use a method reference with the right - * signature - * @return The new cookie for the state if it has changed - */ - final long processModify(final long cookie, final int stateSlot, - @SuppressWarnings("unused") final long unusedModifiedIndex) { - return setFlags(cookie, stateSlot, FLAG_STATE_HAS_MODIFIES); - } - - /** - * Record that a row key has been added to a state, to be applied in - * {@link #applyAddsToStates(ObjectArraySource, ObjectArraySource)}. - * - * @param cookie The last known cookie for the state - * @param stateSlot The state's slot (in main table space) - * @param addedIndex The row key that was added - * @return The new cookie for the state if it has changed - */ - final long processAdd(final long cookie, final int stateSlot, final long addedIndex) { - return setFlagsAndBuild(cookie, stateSlot, (byte) (FLAG_STATE_HAS_ADDS | FLAG_STATE_IN_CURRENT_PASS), - addedIndex); - } - - /** - * Move a state to a new main or overflow table location. - * - * @param cookie The last known cookie for the state - * @param newStateSlot The state's slot (in main table space) - */ - final void processStateMove(final long cookie, final int newStateSlot) { - if (isValidCookie(cookie)) { - final long position = cookieToPosition(cookie); - final long currentSlotAndFlags = updatedStateSlotAndFlags.getLong(position); - final long resultSlotAndFlags = ((long) newStateSlot << FLAG_SHIFT) | (currentSlotAndFlags & FLAG_MASK); - updatedStateSlotAndFlags.set(position, resultSlotAndFlags); - } - } - - private long setFlagsAndBuild(final long cookie, final int stateSlot, final byte flags, final long index) { - final int position; - final long resultCookie; - final long currentSlotAndFlags; - if (isValidCookie(cookie)) { - position = cookieToPosition(cookie); - resultCookie = cookie; - currentSlotAndFlags = updatedStateSlotAndFlags.getLong(position); - } else { - checkCapacity(); - position = size++; - resultCookie = positionToCookie(position); - currentSlotAndFlags = 0L; - } - final RowSetBuilderSequential builder; - final long resultSlotAndFlags = ((long) stateSlot << FLAG_SHIFT) | (currentSlotAndFlags & FLAG_MASK | flags); - if (currentSlotAndFlags != resultSlotAndFlags) { - updatedStateSlotAndFlags.set(position, resultSlotAndFlags); - if ((flags & FLAG_STATE_IN_CURRENT_PASS) != 0 && (currentSlotAndFlags & FLAG_STATE_IN_CURRENT_PASS) == 0) { - checkCurrentPassCapacity(); - currentPassPositions.set(currentPassSize++, position); - } - builders.set(position, builder = RowSetFactory.builderSequential()); - } else { - builder = builders.get(position); - } - // noinspection ConstantConditions - builder.appendKey(index); - return resultCookie; - } - - private long setFlags(final long cookie, final int stateSlot, final byte flags) { - final int position; - final long resultCookie; - final long currentSlotAndFlags; - if (isValidCookie(cookie)) { - position = cookieToPosition(cookie); - resultCookie = cookie; - currentSlotAndFlags = updatedStateSlotAndFlags.getLong(position); - } else { - checkCapacity(); - position = size++; - resultCookie = positionToCookie(position); - currentSlotAndFlags = 0L; - } - final long resultSlotAndFlags = ((long) stateSlot << FLAG_SHIFT) | (currentSlotAndFlags & FLAG_MASK | flags); - if (currentSlotAndFlags != resultSlotAndFlags) { - updatedStateSlotAndFlags.set(position, resultSlotAndFlags); - if ((flags & FLAG_STATE_IN_CURRENT_PASS) != 0 && (currentSlotAndFlags & FLAG_STATE_IN_CURRENT_PASS) == 0) { - checkCurrentPassCapacity(); - currentPassPositions.set(currentPassSize++, position); - } - } - return resultCookie; - } - - private void checkCapacity() { - if (size == capacity) { - capacity += ALLOCATION_UNIT; - updatedStateSlotAndFlags.ensureCapacity(capacity); - builders.ensureCapacity(capacity); - } - } - - private void checkCurrentPassCapacity() { - if (currentPassSize == currentPassCapacity) { - currentPassCapacity += ALLOCATION_UNIT; - currentPassPositions.ensureCapacity(currentPassCapacity); - } - } - - /** - * Apply accumulated adds to their states, populate the result {@link WritableRowRedirection}, and build the initial - * result {@link TrackingWritableRowSet}. - * - * @param rowSetSource The {@link TrackingWritableRowSet} column source for the main table - * @param overflowRowSetSource The {@link TrackingWritableRowSet} column source for the overflow table - * @param rowRedirection The result {@link WritableRowRedirection} (from state first keys to state slots) to - * populate - * @return The result {@link TrackingWritableRowSet} - */ - final TrackingWritableRowSet applyAddsAndMakeInitialRowSet( - @NotNull final ObjectArraySource rowSetSource, - @NotNull final ObjectArraySource overflowRowSetSource, - @NotNull final WritableRowRedirection rowRedirection) { - final RowSetBuilderRandom resultBuilder = RowSetFactory.builderRandom(); - for (long trackerIndex = 0; trackerIndex < size; ++trackerIndex) { - final long slotAndFlags = updatedStateSlotAndFlags.getLong(trackerIndex); - final int slot = (int) (slotAndFlags >> FLAG_SHIFT); - final RowSetBuilderSequential stateBuilder = builders.get(trackerIndex); - builders.set(trackerIndex, null); - - final long stateFirstKey; - // noinspection ConstantConditions - try (final RowSet stateAddedRowSet = stateBuilder.build()) { - final TrackingWritableRowSet stateRowSet = slotToRowSet(rowSetSource, overflowRowSetSource, slot); - stateRowSet.insert(stateAddedRowSet); - stateRowSet.initializePreviousValue(); - stateFirstKey = stateAddedRowSet.firstRowKey(); - } - - rowRedirection.putVoid(stateFirstKey, slot); - resultBuilder.addKey(stateFirstKey); - } - // NB: We should not need to initialize previous value here, as the result RowSet was computed with no - // mutations. - return resultBuilder.build().toTracking(); - } - - /** - * Apply all accumulated removes to this tracker's updated states. - * - * @param rowSetSource The {@link TrackingWritableRowSet} column source for the main table - * @param overflowRowSetSource The {@link TrackingWritableRowSet} column source for the overflow table - */ - final void applyRemovesToStates(@NotNull final ObjectArraySource rowSetSource, - @NotNull final ObjectArraySource overflowRowSetSource) { - for (long trackerIndex = 0; trackerIndex < size; ++trackerIndex) { - final long slotAndFlags = updatedStateSlotAndFlags.getLong(trackerIndex); - // Since removes are always done first, we need not check the flags here. - final int slot = (int) (slotAndFlags >> FLAG_SHIFT); - final RowSetBuilderSequential builder = builders.get(trackerIndex); - builders.set(trackerIndex, null); - - // noinspection ConstantConditions - try (final RowSet stateRemovedRowSet = builder.build()) { - slotToRowSet(rowSetSource, overflowRowSetSource, slot).remove(stateRemovedRowSet); - } - } - } - - /** - * Apply a shift to all "current pass" states. - * - * @param rowSetSource The {@link TrackingWritableRowSet} column source for the main table - * @param overflowRowSetSource The {@link TrackingWritableRowSet} column source for the overflow table - * @param beginRange See {@link RowSetShiftData#applyShift(WritableRowSet, long, long, long)} - * @param endRange See {@link RowSetShiftData#applyShift(WritableRowSet, long, long, long)} - * @param shiftDelta See {@link RowSetShiftData#applyShift(WritableRowSet, long, long, long)} - */ - final void applyShiftToStates(@NotNull final ObjectArraySource rowSetSource, - @NotNull final ObjectArraySource overflowRowSetSource, - final long beginRange, - final long endRange, - final long shiftDelta) { - for (int currentPositionIndex = 0; currentPositionIndex < currentPassSize; ++currentPositionIndex) { - final int trackerIndex = currentPassPositions.getInt(currentPositionIndex); - final long slotAndFlags = updatedStateSlotAndFlags.getLong(trackerIndex); - // Since the current pass is only states responsive to the current shift, we need not check the flags here. - final int slot = (int) (slotAndFlags >> FLAG_SHIFT); - - RowSetShiftData.applyShift(slotToRowSet(rowSetSource, overflowRowSetSource, slot), beginRange, endRange, - shiftDelta); - - updatedStateSlotAndFlags.set(trackerIndex, slotAndFlags ^ FLAG_STATE_IN_CURRENT_PASS); - } - currentPassSize = 0; - } - - /** - * Apply all accumulated adds to this tracker's updated states. - * - * @param rowSetSource The {@link TrackingWritableRowSet} column source for the main table - * @param overflowRowSetSource The {@link TrackingWritableRowSet} column source for the overflow table - */ - final void applyAddsToStates(@NotNull final ObjectArraySource rowSetSource, - @NotNull final ObjectArraySource overflowRowSetSource) { - for (int currentPositionIndex = 0; currentPositionIndex < currentPassSize; ++currentPositionIndex) { - final int trackerIndex = currentPassPositions.getInt(currentPositionIndex); - final long slotAndFlags = updatedStateSlotAndFlags.getLong(trackerIndex); - // Since the current pass is only states with adds, we need not check the flags here. - final int slot = (int) (slotAndFlags >> FLAG_SHIFT); - - final RowSetBuilderSequential builder = builders.get(trackerIndex); - builders.set(trackerIndex, null); - - // noinspection ConstantConditions - try (final RowSet stateAddedRowSet = builder.build()) { - slotToRowSet(rowSetSource, overflowRowSetSource, slot).insert(stateAddedRowSet); - } - - updatedStateSlotAndFlags.set(trackerIndex, slotAndFlags ^ FLAG_STATE_IN_CURRENT_PASS); - } - currentPassSize = 0; - } - - @FunctionalInterface - interface ModifiedColumnSetProducer { - - ModifiedColumnSet produce(boolean someKeyHasAddsOrRemoves, boolean someKeyHasModifies); - } - - /** - * Build an {@link TableUpdateImpl} for this tracker's updated states, and update the result - * {@link TrackingWritableRowSet} and {@link WritableRowRedirection}. - * - * @param rowSetSource The {@link TrackingWritableRowSet} column source for the main table - * @param overflowRowSetSource The {@link TrackingWritableRowSet} column source for the overflow table - * @param rowSet The result {@link TrackingWritableRowSet} of visible keys to update - * @param rowRedirection The result {@link WritableRowRedirection} (from state first keys to state slots) to update - * @param modifiedColumnSetProducer The {@link ModifiedColumnSetProducer} to use for computing the downstream - * {@link ModifiedColumnSet} - * @return The result {@link TableUpdateImpl} - */ - final TableUpdate makeUpdateFromStates(@NotNull final ObjectArraySource rowSetSource, - @NotNull final ObjectArraySource overflowRowSetSource, - @NotNull final WritableRowSet rowSet, - @NotNull final WritableRowRedirection rowRedirection, - @NotNull final ModifiedColumnSetProducer modifiedColumnSetProducer) { - // First pass: Removes are handled on their own, because if the key moved to a new state we may reinsert it - final RowSetBuilderRandom removedBuilder = RowSetFactory.builderRandom(); - int numStatesWithShifts = 0; - for (long ti = 0; ti < size; ++ti) { - final long slotAndFlags = updatedStateSlotAndFlags.getLong(ti); - final byte flags = (byte) (slotAndFlags & FLAG_MASK); - final int slot = (int) (slotAndFlags >> FLAG_SHIFT); - final TrackingRowSet current = slotToRowSet(rowSetSource, overflowRowSetSource, slot); - final long previousFirstKey = current.firstRowKeyPrev(); - if (previousFirstKey == RowSequence.NULL_ROW_KEY) { - // Nothing to remove - continue; - } - if (current.isEmpty()) { - // We must have removed everything - rowRedirection.removeVoid(previousFirstKey); - removedBuilder.addKey(previousFirstKey); - continue; - } - final long currentFirstKey = current.firstRowKey(); - if (previousFirstKey != currentFirstKey) { - // First key changed - rowRedirection.removeVoid(previousFirstKey); - if (flags == FLAG_STATE_HAS_SHIFTS) { - ++numStatesWithShifts; - } else { - // Not just a shift - removedBuilder.addKey(previousFirstKey); - } - } - } - - // Second pass: Everything else - final RowSetBuilderRandom addedBuilder = RowSetFactory.builderRandom(); - final RowSetBuilderRandom modifiedBuilder = RowSetFactory.builderRandom(); - boolean someKeyHasAddsOrRemoves = false; - boolean someKeyHasModifies = false; - final RowSetShiftData shiftData; - try (final WritableLongChunk previousShiftedFirstKeys = - WritableLongChunk.makeWritableChunk(numStatesWithShifts); - final WritableLongChunk currentShiftedFirstKeys = - WritableLongChunk.makeWritableChunk(numStatesWithShifts)) { - int shiftChunkPosition = 0; - for (long ti = 0; ti < size; ++ti) { - final long slotAndFlags = updatedStateSlotAndFlags.getLong(ti); - final byte flags = (byte) (slotAndFlags & FLAG_MASK); - final int slot = (int) (slotAndFlags >> FLAG_SHIFT); - final TrackingRowSet current = slotToRowSet(rowSetSource, overflowRowSetSource, slot); - if (current.isEmpty()) { - // Removes are already handled - continue; - } - final long previousFirstKey = current.firstRowKeyPrev(); - final long currentFirstKey = current.firstRowKey(); - if (previousFirstKey == RowSequence.NULL_ROW_KEY) { - // We must have added something - rowRedirection.putVoid(currentFirstKey, slot); - addedBuilder.addKey(currentFirstKey); - continue; - } - if (previousFirstKey == currentFirstKey) { - if (flags != FLAG_STATE_HAS_SHIFTS) { - // @formatter:off - someKeyHasAddsOrRemoves |= ((flags & (FLAG_STATE_HAS_REMOVES | FLAG_STATE_HAS_ADDS)) != 0); - someKeyHasModifies |= ((flags & FLAG_STATE_HAS_MODIFIES ) != 0); - // @formatter:on - modifiedBuilder.addKey(currentFirstKey); - } - } else { - rowRedirection.putVoid(currentFirstKey, slot); - if (flags == FLAG_STATE_HAS_SHIFTS) { - previousShiftedFirstKeys.set(shiftChunkPosition, previousFirstKey); - currentShiftedFirstKeys.set(shiftChunkPosition, currentFirstKey); - ++shiftChunkPosition; - } else { - addedBuilder.addKey(currentFirstKey); - } - } - } - - // Now sort shifts and build the shift data - Assert.eq(numStatesWithShifts, "numStatesWithShift", shiftChunkPosition, "shiftedChunkPosition"); - if (numStatesWithShifts > 0) { - previousShiftedFirstKeys.setSize(numStatesWithShifts); - currentShiftedFirstKeys.setSize(numStatesWithShifts); - try (final LongLongTimsortKernel.LongLongSortKernelContext sortKernelContext = - LongLongTimsortKernel.createContext(numStatesWithShifts)) { - LongLongTimsortKernel.sort(sortKernelContext, currentShiftedFirstKeys, previousShiftedFirstKeys); - } - final RowSetShiftData.Builder shiftBuilder = new RowSetShiftData.Builder(); - for (int si = 0; si < numStatesWithShifts; ++si) { - final long previousKey = previousShiftedFirstKeys.get(si); - final long currentKey = currentShiftedFirstKeys.get(si); - shiftBuilder.shiftRange(previousKey, previousKey, currentKey - previousKey); - } - shiftData = shiftBuilder.build(); - } else { - shiftData = RowSetShiftData.EMPTY; - } - } - - // Build the notification indexes - final RowSet added = addedBuilder.build(); - final RowSet removed = removedBuilder.build(); - final RowSet modified = modifiedBuilder.build(); - - // Update the result TrackingWritableRowSet - rowSet.remove(removed); - shiftData.apply(rowSet); - rowSet.insert(added); - - // Build and return the update - return new TableUpdateImpl(added, removed, modified, shiftData, - modifiedColumnSetProducer.produce(someKeyHasAddsOrRemoves, someKeyHasModifies)); - } - - private static TrackingWritableRowSet slotToRowSet( - @NotNull final ObjectArraySource rowSetSource, - @NotNull final ObjectArraySource overflowRowSetSource, - final int slot) { - return IncrementalChunkedByAggregationStateManager.isOverflowLocation(slot) - ? overflowRowSetSource - .get(IncrementalChunkedByAggregationStateManager.hashLocationToOverflowLocation(slot)) - : rowSetSource.get(slot); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.java deleted file mode 100644 index 153bbe1c494..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.java +++ /dev/null @@ -1,1697 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.base.verify.Require; -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.Any; -import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.chunk.attributes.HashCodes; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.table.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.util.QueryConstants; -import io.deephaven.chunk.util.hashing.*; -// this is ugly to have twice, but we do need it twice for replication -// @StateChunkIdentityName@ from \QObjectChunkIdentity\E -import io.deephaven.chunk.util.hashing.ObjectChunkIdentityEquals; -import io.deephaven.engine.table.impl.sort.permute.PermuteKernel; -import io.deephaven.engine.table.impl.sort.timsort.LongIntTimsortKernel; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.*; - -// mixin rehash -import java.util.Arrays; -import io.deephaven.engine.table.impl.sort.permute.IntPermuteKernel; -// @StateChunkTypeEnum@ from \QObject\E -import io.deephaven.engine.table.impl.sort.permute.ObjectPermuteKernel; -import io.deephaven.engine.table.impl.util.compact.IntCompactKernel; -import io.deephaven.engine.table.impl.util.compact.LongCompactKernel; -// endmixin rehash - -import io.deephaven.util.SafeCloseableArray; -import org.jetbrains.annotations.NotNull; - -// region extra imports -import io.deephaven.engine.table.impl.HashTableAnnotations; -import io.deephaven.engine.table.impl.PrevColumnSource; - -import java.util.function.UnaryOperator; - -import static io.deephaven.engine.table.impl.by.IncrementalByAggregationUpdateTracker.NULL_COOKIE; -// endregion extra imports - -import static io.deephaven.util.SafeCloseable.closeArray; - -// region class visibility -public -// endregion class visibility -class IncrementalChunkedByAggregationStateManager - // region extensions - // endregion extensions -{ - // region constants - static final int CHUNK_SIZE = 1 << 12; - static final int MINIMUM_INITIAL_HASH_SIZE = 1 << 11; // io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE - static final int MAX_TABLE_SIZE = HashTableColumnSource.MINIMUM_OVERFLOW_HASH_SLOT; - // endregion constants - - // mixin rehash - static final double DEFAULT_MAX_LOAD_FACTOR = 0.75; - static final double DEFAULT_TARGET_LOAD_FACTOR = 0.70; - // endmixin rehash - - // region preamble variables - @FunctionalInterface - interface StateTrackingCallback { - - /** - * Invoke a callback that will allow external trackers to record changes to states in build or probe calls. - * - * @param cookie The current cookie for the state, or 0 on insert - * @param stateSlot The state slot (in main table space) - * @param index The built or probed row key - * @return The new cookie for the state - */ - long invoke(long cookie, int stateSlot, long index); - } - - @FunctionalInterface - interface ShiftAppliedCallback { - - /** - * Invoke a callback that will allow external trackers to record an applied shift. - * - * @param cookie The current cookie for the state, or 0 on insert - * @param stateSlot The state slot (in main table space) - * @return The new cookie for the state - */ - long invoke(long cookie, int stateSlot); - } - - @FunctionalInterface - interface StateMoveCallback { - - /** - * Invoke a callback that will allow external trackers to record a state's move from one slot to another on - * rehash. - * - * @param cookie The current cookie for the state - * @param newStateSlot The new state slot (in main table space) - */ - void invoke(long cookie, int newStateSlot); - } - // endregion preamble variables - - @HashTableAnnotations.EmptyStateValue - // @NullStateValue@ from \Qnull\E, @StateValueType@ from \QTrackingWritableRowSet\E - private static final TrackingWritableRowSet EMPTY_VALUE = null; - - // mixin getStateValue - // region overflow pivot - static final int MINIMUM_OVERFLOW_HASH_SLOT = MAX_TABLE_SIZE; - // endregion overflow pivot - // endmixin getStateValue - - // the number of slots in our table - // mixin rehash - private int tableSize; - // endmixin rehash - // altmixin rehash: private final int tableSize; - - // how many key columns we have - private final int keyColumnCount; - - // mixin rehash - private long numEntries = 0; - - /** Our table size must be 2^L (i.e. a power of two); and the pivot is between 2^(L-1) and 2^L. - * - *

When hashing a value, if hashCode % 2^L < tableHashPivot; then the destination location is hashCode % 2^L. - * If hashCode % 2^L >= tableHashPivot, then the destination location is hashCode % 2^(L-1). Once the pivot reaches - * the table size, we can simply double the table size and repeat the process.

- * - *

This has the effect of only using hash table locations < hashTablePivot. When we want to expand the table - * we can move some of the entries from the location {@code tableHashPivot - 2^(L-1)} to tableHashPivot. This - * provides for incremental expansion of the hash table, without the need for a full rehash.

- */ - private int tableHashPivot; - - // the table will be rehashed to a load factor of targetLoadFactor if our loadFactor exceeds maximumLoadFactor - // or if it falls below minimum load factor we will instead contract the table - private double targetLoadFactor = DEFAULT_TARGET_LOAD_FACTOR; - private double maximumLoadFactor = DEFAULT_MAX_LOAD_FACTOR; - // TODO: We do not yet support contraction - // private final double minimumLoadFactor = 0.5; - - private final IntegerArraySource freeOverflowLocations = new IntegerArraySource(); - private int freeOverflowCount = 0; - // endmixin rehash - - // the keys for our hash entries - private final ArrayBackedColumnSource[] keySources; - // the location of any overflow entry in this bucket - private final IntegerArraySource overflowLocationSource = new IntegerArraySource(); - - // we are going to also reuse this for our state entry, so that we do not need additional storage - @HashTableAnnotations.StateColumnSource - // @StateColumnSourceType@ from \QObjectArraySource\E - private final ObjectArraySource rowSetSource - // @StateColumnSourceConstructor@ from \QObjectArraySource<>(TrackingWritableRowSet.class)\E - = new ObjectArraySource<>(TrackingWritableRowSet.class); - - // the keys for overflow - private int nextOverflowLocation = 0; - private final ArrayBackedColumnSource [] overflowKeySources; - // the location of the next key in an overflow bucket - private final IntegerArraySource overflowOverflowLocationSource = new IntegerArraySource(); - // the overflow buckets for the state source - @HashTableAnnotations.OverflowStateColumnSource - // @StateColumnSourceType@ from \QObjectArraySource\E - private final ObjectArraySource overflowRowSetSource - // @StateColumnSourceConstructor@ from \QObjectArraySource<>(TrackingWritableRowSet.class)\E - = new ObjectArraySource<>(TrackingWritableRowSet.class); - - // the type of each of our key chunks - private final ChunkType[] keyChunkTypes; - - // the operators for hashing and various equality methods - private final ChunkHasher[] chunkHashers; - private final ChunkEquals[] chunkEquals; - private final PermuteKernel[] chunkCopiers; - - // mixin rehash - // If we have objects in our key columns, then we should null them out if we delete an overflow row, this only - // applies to ObjectArraySources, for primitives we are content to leave the dead entries in the tables, because - // they will not affect GC. - private final ObjectArraySource[] overflowKeyColumnsToNull; - // endmixin rehash - - // region extra variables - // These are re-used in all build and probe passes for interaction with the IncrementalByAggregationUpdateTracker - private final LongArraySource cookieSource = new LongArraySource(); - private final LongArraySource overflowCookieSource = new LongArraySource(); - // endregion extra variables - - IncrementalChunkedByAggregationStateManager(ColumnSource[] tableKeySources - , int tableSize - // region constructor arguments - , double targetLoadFactor - , double maximumLoadFactor - // endregion constructor arguments - ) { - // region super - // endregion super - keyColumnCount = tableKeySources.length; - - this.tableSize = tableSize; - Require.leq(tableSize, "tableSize", MAX_TABLE_SIZE); - Require.gtZero(tableSize, "tableSize"); - Require.eq(Integer.bitCount(tableSize), "Integer.bitCount(tableSize)", 1); - // mixin rehash - this.tableHashPivot = tableSize; - // endmixin rehash - - overflowKeySources = new ArrayBackedColumnSource[keyColumnCount]; - keySources = new ArrayBackedColumnSource[keyColumnCount]; - - keyChunkTypes = new ChunkType[keyColumnCount]; - chunkHashers = new ChunkHasher[keyColumnCount]; - chunkEquals = new ChunkEquals[keyColumnCount]; - chunkCopiers = new PermuteKernel[keyColumnCount]; - - for (int ii = 0; ii < keyColumnCount; ++ii) { - // the sources that we will use to store our hash table - keySources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(tableSize, tableKeySources[ii].getType()); - keyChunkTypes[ii] = tableKeySources[ii].getChunkType(); - - overflowKeySources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(CHUNK_SIZE, tableKeySources[ii].getType()); - - chunkHashers[ii] = ChunkHasher.makeHasher(keyChunkTypes[ii]); - chunkEquals[ii] = ChunkEquals.makeEqual(keyChunkTypes[ii]); - chunkCopiers[ii] = PermuteKernel.makePermuteKernel(keyChunkTypes[ii]); - } - - // mixin rehash - overflowKeyColumnsToNull = Arrays.stream(overflowKeySources).filter(x -> x instanceof ObjectArraySource).map(x -> (ObjectArraySource)x).toArray(ObjectArraySource[]::new); - // endmixin rehash - - // region constructor - setTargetLoadFactor(targetLoadFactor); - setMaximumLoadFactor(maximumLoadFactor); - // endregion constructor - - ensureCapacity(tableSize); - } - - private void ensureCapacity(int tableSize) { - rowSetSource.ensureCapacity(tableSize); - overflowLocationSource.ensureCapacity(tableSize); - for (int ii = 0; ii < keyColumnCount; ++ii) { - keySources[ii].ensureCapacity(tableSize); - } - // region ensureCapacity - cookieSource.ensureCapacity(tableSize); - // endregion ensureCapacity - } - - private void ensureOverflowCapacity(WritableIntChunk chunkPositionsToInsertInOverflow) { - final int locationsToAllocate = chunkPositionsToInsertInOverflow.size(); - // mixin rehash - if (freeOverflowCount >= locationsToAllocate) { - return; - } - final int newCapacity = nextOverflowLocation + locationsToAllocate - freeOverflowCount; - // endmixin rehash - // altmixin rehash: final int newCapacity = nextOverflowLocation + locationsToAllocate; - overflowOverflowLocationSource.ensureCapacity(newCapacity); - overflowRowSetSource.ensureCapacity(newCapacity); - //noinspection ForLoopReplaceableByForEach - for (int ii = 0; ii < overflowKeySources.length; ++ii) { - overflowKeySources[ii].ensureCapacity(newCapacity); - } - // region ensureOverflowCapacity - overflowCookieSource.ensureCapacity(newCapacity); - // endregion ensureOverflowCapacity - } - - // region build wrappers - void buildInitialTableFromCurrent(@NotNull final Table sourceTable, @NotNull final ColumnSource[] keySources, @NotNull final IncrementalByAggregationUpdateTracker aggregationUpdateTracker) { - if (sourceTable.isEmpty()) { - return; - } - try (final BuildContext bc = makeBuildContext(keySources, sourceTable.size())) { - buildTable(bc, sourceTable.getRowSet(), keySources, aggregationUpdateTracker::processInitialAdd, aggregationUpdateTracker::processStateMove); - } - } - - void buildInitialTableFromPrevious(@NotNull final Table sourceTable, @NotNull final ColumnSource[] keySources, @NotNull final IncrementalByAggregationUpdateTracker aggregationUpdateTracker) { - final ColumnSource[] prevKeySources; - try (final RowSet prevRowSet = sourceTable.getRowSet().copyPrev()) { - if (prevRowSet.isEmpty()) { - return; - } - prevKeySources = Arrays.stream(keySources).map((UnaryOperator>) PrevColumnSource::new).toArray(ColumnSource[]::new); - try (final BuildContext bc = makeBuildContext(prevKeySources, prevRowSet.size())) { - buildTable(bc, prevRowSet, prevKeySources, aggregationUpdateTracker::processInitialAdd, aggregationUpdateTracker::processStateMove); - } - } - } - - void processAdds(@NotNull final ColumnSource[] keySources, @NotNull final RowSet addedRowSet, @NotNull final IncrementalByAggregationUpdateTracker aggregationUpdateTracker) { - if (addedRowSet.isEmpty()) { - return; - } - try (final BuildContext bc = makeBuildContext(keySources, addedRowSet.size())) { - buildTable(bc, addedRowSet, keySources, aggregationUpdateTracker::processAdd, aggregationUpdateTracker::processStateMove); - } - } - // endregion build wrappers - - class BuildContext implements Context { - final int chunkSize; - - final LongIntTimsortKernel.LongIntSortKernelContext sortContext; - final ColumnSource.FillContext stateSourceFillContext; - // mixin rehash - final ColumnSource.FillContext overflowStateSourceFillContext; - // endmixin rehash - final ColumnSource.FillContext overflowFillContext; - final ColumnSource.FillContext overflowOverflowFillContext; - - // the chunk of hashcodes - final WritableIntChunk hashChunk; - // the chunk of positions within our table - final WritableLongChunk tableLocationsChunk; - - final ResettableWritableChunk[] writeThroughChunks = getResettableWritableKeyChunks(); - final WritableIntChunk sourcePositions; - final WritableIntChunk destinationLocationPositionInWriteThrough; - - final WritableBooleanChunk filledValues; - final WritableBooleanChunk equalValues; - - // the overflow locations that we need to get from the overflowLocationSource (or overflowOverflowLocationSource) - final WritableLongChunk overflowLocationsToFetch; - // the overflow position in the working key chunks, parallel to the overflowLocationsToFetch - final WritableIntChunk overflowPositionInSourceChunk; - - // the position with our hash table that we should insert a value into - final WritableLongChunk insertTableLocations; - // the position in our chunk, parallel to the workingChunkInsertTablePositions - final WritableIntChunk insertPositionsInSourceChunk; - - // we sometimes need to check two positions within a single chunk for equality, this contains those positions as pairs - final WritableIntChunk chunkPositionsToCheckForEquality; - // While processing overflow insertions, parallel to the chunkPositions to check for equality, the overflow location that - // is represented by the first of the pairs in chunkPositionsToCheckForEquality - final WritableLongChunk overflowLocationForEqualityCheck; - - // the chunk of state values that we read from the hash table - // @WritableStateChunkType@ from \QWritableObjectChunk\E - final WritableObjectChunk workingStateEntries; - - // the chunks for getting key values from the hash table - final WritableChunk[] workingKeyChunks; - final WritableChunk[] overflowKeyChunks; - - // when fetching from the overflow, we record which chunk position we are fetching for - final WritableIntChunk chunkPositionsForFetches; - // which positions in the chunk we are inserting into the overflow - final WritableIntChunk chunkPositionsToInsertInOverflow; - // which table locations we are inserting into the overflow - final WritableLongChunk tableLocationsToInsertInOverflow; - - // values we have read from the overflow locations sources - final WritableIntChunk overflowLocations; - - // mixin rehash - final WritableLongChunk rehashLocations; - final WritableIntChunk overflowLocationsToMigrate; - final WritableLongChunk overflowLocationsAsKeyIndices; - final WritableBooleanChunk shouldMoveBucket; - - final ResettableWritableLongChunk overflowLocationForPromotionLoop = ResettableWritableLongChunk.makeResettableChunk(); - - final ResettableWritableIntChunk writeThroughOverflowLocations = ResettableWritableIntChunk.makeResettableChunk(); - // endmixin rehash - - final SharedContext sharedFillContext; - final ColumnSource.FillContext[] workingFillContexts; - final SharedContext sharedOverflowContext; - final ColumnSource.FillContext[] overflowContexts; - final SharedContext sharedBuildContext; - final ChunkSource.GetContext[] buildContexts; - - // region build context - // endregion build context - - final boolean haveSharedContexts; - - private BuildContext(ColumnSource[] buildSources, - int chunkSize - // region build context constructor args - // endregion build context constructor args - ) { - Assert.gtZero(chunkSize, "chunkSize"); - this.chunkSize = chunkSize; - haveSharedContexts = buildSources.length > 1; - if (haveSharedContexts) { - sharedFillContext = SharedContext.makeSharedContext(); - sharedOverflowContext = SharedContext.makeSharedContext(); - sharedBuildContext = SharedContext.makeSharedContext(); - } else { - // no point in the additional work implied by these not being null. - sharedFillContext = null; - sharedOverflowContext = null; - sharedBuildContext = null; - } - workingFillContexts = makeFillContexts(keySources, sharedFillContext, chunkSize); - overflowContexts = makeFillContexts(overflowKeySources, sharedOverflowContext, chunkSize); - buildContexts = makeGetContexts(buildSources, sharedBuildContext, chunkSize); - // region build context constructor - // endregion build context constructor - sortContext = LongIntTimsortKernel.createContext(chunkSize); - stateSourceFillContext = rowSetSource.makeFillContext(chunkSize); - overflowFillContext = overflowLocationSource.makeFillContext(chunkSize); - overflowOverflowFillContext = overflowOverflowLocationSource.makeFillContext(chunkSize); - hashChunk = WritableIntChunk.makeWritableChunk(chunkSize); - tableLocationsChunk = WritableLongChunk.makeWritableChunk(chunkSize); - sourcePositions = WritableIntChunk.makeWritableChunk(chunkSize); - destinationLocationPositionInWriteThrough = WritableIntChunk.makeWritableChunk(chunkSize); - filledValues = WritableBooleanChunk.makeWritableChunk(chunkSize); - equalValues = WritableBooleanChunk.makeWritableChunk(chunkSize); - overflowLocationsToFetch = WritableLongChunk.makeWritableChunk(chunkSize); - overflowPositionInSourceChunk = WritableIntChunk.makeWritableChunk(chunkSize); - insertTableLocations = WritableLongChunk.makeWritableChunk(chunkSize); - insertPositionsInSourceChunk = WritableIntChunk.makeWritableChunk(chunkSize); - chunkPositionsToCheckForEquality = WritableIntChunk.makeWritableChunk(chunkSize * 2); - overflowLocationForEqualityCheck = WritableLongChunk.makeWritableChunk(chunkSize); - // @WritableStateChunkName@ from \QWritableObjectChunk\E - workingStateEntries = WritableObjectChunk.makeWritableChunk(chunkSize); - workingKeyChunks = getWritableKeyChunks(chunkSize); - overflowKeyChunks = getWritableKeyChunks(chunkSize); - chunkPositionsForFetches = WritableIntChunk.makeWritableChunk(chunkSize); - chunkPositionsToInsertInOverflow = WritableIntChunk.makeWritableChunk(chunkSize); - tableLocationsToInsertInOverflow = WritableLongChunk.makeWritableChunk(chunkSize); - overflowLocations = WritableIntChunk.makeWritableChunk(chunkSize); - // mixin rehash - rehashLocations = WritableLongChunk.makeWritableChunk(chunkSize); - overflowStateSourceFillContext = overflowRowSetSource.makeFillContext(chunkSize); - overflowLocationsToMigrate = WritableIntChunk.makeWritableChunk(chunkSize); - overflowLocationsAsKeyIndices = WritableLongChunk.makeWritableChunk(chunkSize); - shouldMoveBucket = WritableBooleanChunk.makeWritableChunk(chunkSize); - // endmixin rehash - } - - private void resetSharedContexts() { - if (!haveSharedContexts) { - return; - } - sharedFillContext.reset(); - sharedOverflowContext.reset(); - sharedBuildContext.reset(); - } - - private void closeSharedContexts() { - if (!haveSharedContexts) { - return; - } - sharedFillContext.close(); - sharedOverflowContext.close(); - sharedBuildContext.close(); - } - - @Override - public void close() { - sortContext.close(); - stateSourceFillContext.close(); - // mixin rehash - overflowStateSourceFillContext.close(); - // endmixin rehash - overflowFillContext.close(); - overflowOverflowFillContext.close(); - closeArray(workingFillContexts); - closeArray(overflowContexts); - closeArray(buildContexts); - - hashChunk.close(); - tableLocationsChunk.close(); - closeArray(writeThroughChunks); - - sourcePositions.close(); - destinationLocationPositionInWriteThrough.close(); - filledValues.close(); - equalValues.close(); - overflowLocationsToFetch.close(); - overflowPositionInSourceChunk.close(); - insertTableLocations.close(); - insertPositionsInSourceChunk.close(); - chunkPositionsToCheckForEquality.close(); - overflowLocationForEqualityCheck.close(); - workingStateEntries.close(); - closeArray(workingKeyChunks); - closeArray(overflowKeyChunks); - chunkPositionsForFetches.close(); - chunkPositionsToInsertInOverflow.close(); - tableLocationsToInsertInOverflow.close(); - overflowLocations.close(); - // mixin rehash - rehashLocations.close(); - overflowLocationsToMigrate.close(); - overflowLocationsAsKeyIndices.close(); - shouldMoveBucket.close(); - overflowLocationForPromotionLoop.close(); - writeThroughOverflowLocations.close(); - // endmixin rehash - // region build context close - // endregion build context close - closeSharedContexts(); - } - - } - - BuildContext makeBuildContext(ColumnSource[] buildSources, - long maxSize - // region makeBuildContext args - // endregion makeBuildContext args - ) { - return new BuildContext(buildSources, (int)Math.min(CHUNK_SIZE, maxSize) - // region makeBuildContext arg pass - // endregion makeBuildContext arg pass - ); - } - - private void buildTable(final BuildContext bc, - final RowSequence buildIndex, - ColumnSource[] buildSources - // region extra build arguments - , final StateTrackingCallback trackingCallback - , final StateMoveCallback moveCallback - // endregion extra build arguments - ) { - long hashSlotOffset = 0; - // region build start - // endregion build start - - try (final RowSequence.Iterator rsIt = buildIndex.getRowSequenceIterator(); - // region build initialization try - // endregion build initialization try - ) { - // region build initialization - // Row keys extracted from the input RowSequence, parallel to the sourceKeyChunks - final WritableLongChunk sourceChunkIndexKeys = WritableLongChunk.makeWritableChunk(bc.chunkSize); - // endregion build initialization - - // chunks to write through to the table key sources - - - //noinspection unchecked - final Chunk [] sourceKeyChunks = new Chunk[buildSources.length]; - - while (rsIt.hasMore()) { - // we reset early to avoid carrying around state for old RowSequence which can't be reused. - bc.resetSharedContexts(); - - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(bc.chunkSize); - - getKeyChunks(buildSources, bc.buildContexts, sourceKeyChunks, chunkOk); - hashKeyChunks(bc.hashChunk, sourceKeyChunks); - - // region build loop initialization - chunkOk.fillRowKeyChunk(sourceChunkIndexKeys); - // endregion build loop initialization - - // turn hash codes into indices within our table - convertHashToTableLocations(bc.hashChunk, bc.tableLocationsChunk); - - // now fetch the values from the table, note that we do not order these fetches - fillKeys(bc.workingFillContexts, bc.workingKeyChunks, bc.tableLocationsChunk); - - // and the corresponding states, if a value is null, we've found our insertion point - rowSetSource.fillChunkUnordered(bc.stateSourceFillContext, bc.workingStateEntries, bc.tableLocationsChunk); - - // find things that exist - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(bc.workingStateEntries, EMPTY_VALUE, bc.filledValues); - - // to be equal, the location must exist; and each of the keyChunks must match - bc.equalValues.setSize(bc.filledValues.size()); - bc.equalValues.copyFromChunk(bc.filledValues, 0, 0, bc.filledValues.size()); - checkKeyEquality(bc.equalValues, bc.workingKeyChunks, sourceKeyChunks); - - bc.overflowPositionInSourceChunk.setSize(0); - bc.overflowLocationsToFetch.setSize(0); - bc.insertPositionsInSourceChunk.setSize(0); - bc.insertTableLocations.setSize(0); - - for (int ii = 0; ii < bc.equalValues.size(); ++ii) { - final long tableLocation = bc.tableLocationsChunk.get(ii); - if (bc.equalValues.get(ii)) { - // region build found main - cookieSource.set(tableLocation, trackingCallback.invoke(cookieSource.getLong(tableLocation), (int) tableLocation, sourceChunkIndexKeys.get(ii))); - // endregion build found main - } else if (bc.filledValues.get(ii)) { - // we must handle this as part of the overflow bucket - bc.overflowPositionInSourceChunk.add(ii); - bc.overflowLocationsToFetch.add(tableLocation); - } else { - // for the values that are empty, we record them in the insert chunks - bc.insertPositionsInSourceChunk.add(ii); - bc.insertTableLocations.add(tableLocation); - } - } - - // we first sort by position; so that we'll not insert things into the table twice or overwrite - // collisions - LongIntTimsortKernel.sort(bc.sortContext, bc.insertPositionsInSourceChunk, bc.insertTableLocations); - - // the first and last valid table location in our writeThroughChunks - long firstBackingChunkLocation = -1; - long lastBackingChunkLocation = -1; - - bc.chunkPositionsToCheckForEquality.setSize(0); - bc.destinationLocationPositionInWriteThrough.setSize(0); - bc.sourcePositions.setSize(0); - - for (int ii = 0; ii < bc.insertPositionsInSourceChunk.size(); ) { - final int firstChunkPositionForHashLocation = bc.insertPositionsInSourceChunk.get(ii); - final long currentHashLocation = bc.insertTableLocations.get(ii); - - // region main insert - rowSetSource.set(currentHashLocation, RowSetFactory.empty().toTracking()); - cookieSource.set(currentHashLocation, trackingCallback.invoke(NULL_COOKIE, (int) currentHashLocation, sourceChunkIndexKeys.get(firstChunkPositionForHashLocation))); - // endregion main insert - // mixin rehash - numEntries++; - // endmixin rehash - - if (currentHashLocation > lastBackingChunkLocation) { - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, currentHashLocation, keySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - } - - bc.sourcePositions.add(firstChunkPositionForHashLocation); - bc.destinationLocationPositionInWriteThrough.add((int)(currentHashLocation - firstBackingChunkLocation)); - - final int currentHashValue = bc.hashChunk.get(firstChunkPositionForHashLocation); - - while (++ii < bc.insertTableLocations.size() && bc.insertTableLocations.get(ii) == currentHashLocation) { - // if this thing is equal to the first one; we should mark the appropriate slot, we don't - // know the types and don't want to make the virtual calls, so we need to just accumulate - // the things to check for equality afterwards - final int chunkPosition = bc.insertPositionsInSourceChunk.get(ii); - if (bc.hashChunk.get(chunkPosition) != currentHashValue) { - // we must be an overflow - bc.overflowPositionInSourceChunk.add(chunkPosition); - bc.overflowLocationsToFetch.add(currentHashLocation); - } else { - // we need to check equality, equal things are the same slot; unequal things are overflow - bc.chunkPositionsToCheckForEquality.add(firstChunkPositionForHashLocation); - bc.chunkPositionsToCheckForEquality.add(chunkPosition); - } - } - } - - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - - checkPairEquality(bc.chunkPositionsToCheckForEquality, sourceKeyChunks, bc.equalValues); - - for (int ii = 0; ii < bc.equalValues.size(); ii++) { - final int chunkPosition = bc.chunkPositionsToCheckForEquality.get(ii * 2 + 1); - final long tableLocation = bc.tableLocationsChunk.get(chunkPosition); - - if (bc.equalValues.get(ii)) { - // region build main duplicate - // NB: We just inserted this slot, so there's no way that its cookie can have changed. - trackingCallback.invoke(cookieSource.getLong(tableLocation), (int) tableLocation, sourceChunkIndexKeys.get(chunkPosition)); - // endregion build main duplicate - } else { - // we are an overflow element - bc.overflowPositionInSourceChunk.add(chunkPosition); - bc.overflowLocationsToFetch.add(tableLocation); - } - } - - // now handle overflow - if (bc.overflowPositionInSourceChunk.size() > 0) { - // on the first pass we fill from the table's locations - overflowLocationSource.fillChunkUnordered(bc.overflowFillContext, bc.overflowLocations, bc.overflowLocationsToFetch); - bc.chunkPositionsToInsertInOverflow.setSize(0); - bc.tableLocationsToInsertInOverflow.setSize(0); - - // overflow slots now contains the positions in the overflow columns - - while (bc.overflowPositionInSourceChunk.size() > 0) { - // now we have the overflow slot for each of the things we are interested in. - // if the slot is null, then we can insert it and we are complete. - - bc.overflowLocationsToFetch.setSize(0); - bc.chunkPositionsForFetches.setSize(0); - - // TODO: Crunch it down - for (int ii = 0; ii < bc.overflowLocations.size(); ++ii) { - final int overflowLocation = bc.overflowLocations.get(ii); - final int chunkPosition = bc.overflowPositionInSourceChunk.get(ii); - if (overflowLocation == QueryConstants.NULL_INT) { - // insert me into overflow in the next free overflow slot - bc.chunkPositionsToInsertInOverflow.add(chunkPosition); - bc.tableLocationsToInsertInOverflow.add(bc.tableLocationsChunk.get(chunkPosition)); - } else { - // add to the key positions to fetch - bc.chunkPositionsForFetches.add(chunkPosition); - bc.overflowLocationsToFetch.add(overflowLocation); - } - } - - // if the slot is non-null, then we need to fetch the overflow values for comparison - fillOverflowKeys(bc.overflowContexts, bc.overflowKeyChunks, bc.overflowLocationsToFetch); - - // now compare the value in our overflowKeyChunk to the value in the sourceChunk - checkLhsPermutedEquality(bc.chunkPositionsForFetches, sourceKeyChunks, bc.overflowKeyChunks, bc.equalValues); - - int writePosition = 0; - for (int ii = 0; ii < bc.equalValues.size(); ++ii) { - final int chunkPosition = bc.chunkPositionsForFetches.get(ii); - final long overflowLocation = bc.overflowLocationsToFetch.get(ii); - if (bc.equalValues.get(ii)) { - // region build overflow found - overflowCookieSource.set(overflowLocation, trackingCallback.invoke(overflowCookieSource.getLong(overflowLocation), overflowLocationToHashLocation((int) overflowLocation), sourceChunkIndexKeys.get(chunkPosition))); - // endregion build overflow found - } else { - // otherwise, we need to repeat the overflow calculation, with our next overflow fetch - bc.overflowLocationsToFetch.set(writePosition, overflowLocation); - bc.overflowPositionInSourceChunk.set(writePosition++, chunkPosition); - } - } - bc.overflowLocationsToFetch.setSize(writePosition); - bc.overflowPositionInSourceChunk.setSize(writePosition); - - // on subsequent iterations, we are following the overflow chains, so we fill from the overflowOverflowLocationSource - if (bc.overflowPositionInSourceChunk.size() > 0) { - overflowOverflowLocationSource.fillChunkUnordered(bc.overflowOverflowFillContext, bc.overflowLocations, bc.overflowLocationsToFetch); - } - } - - // make sure we actually have enough room to insert stuff where we would like - ensureOverflowCapacity(bc.chunkPositionsToInsertInOverflow); - - firstBackingChunkLocation = -1; - lastBackingChunkLocation = -1; - bc.destinationLocationPositionInWriteThrough.setSize(0); - bc.sourcePositions.setSize(0); - - // do the overflow insertions, one per table position at a time; until we have no insertions left - while (bc.chunkPositionsToInsertInOverflow.size() > 0) { - // sort by table position - LongIntTimsortKernel.sort(bc.sortContext, bc.chunkPositionsToInsertInOverflow, bc.tableLocationsToInsertInOverflow); - - bc.chunkPositionsToCheckForEquality.setSize(0); - bc.overflowLocationForEqualityCheck.setSize(0); - - for (int ii = 0; ii < bc.chunkPositionsToInsertInOverflow.size(); ) { - final long tableLocation = bc.tableLocationsToInsertInOverflow.get(ii); - final int chunkPosition = bc.chunkPositionsToInsertInOverflow.get(ii); - - final int allocatedOverflowLocation = allocateOverflowLocation(); - - // we are inserting into the head of the list, so we move the existing overflow into our overflow - overflowOverflowLocationSource.set(allocatedOverflowLocation, overflowLocationSource.getUnsafe(tableLocation)); - // and we point the overflow at our slot - overflowLocationSource.set(tableLocation, allocatedOverflowLocation); - - // region build overflow insert - overflowRowSetSource.set(allocatedOverflowLocation, RowSetFactory.empty().toTracking()); - overflowCookieSource.set(allocatedOverflowLocation, trackingCallback.invoke(NULL_COOKIE, overflowLocationToHashLocation(allocatedOverflowLocation), sourceChunkIndexKeys.get(chunkPosition))); - // endregion build overflow insert - - // mixin rehash - numEntries++; - // endmixin rehash - - // get the backing chunk from the overflow keys - if (allocatedOverflowLocation > lastBackingChunkLocation || allocatedOverflowLocation < firstBackingChunkLocation) { - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, allocatedOverflowLocation, overflowKeySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - } - - // now we must set all of our key values in the overflow - bc.sourcePositions.add(chunkPosition); - bc.destinationLocationPositionInWriteThrough.add((int)(allocatedOverflowLocation - firstBackingChunkLocation)); - - while (++ii < bc.tableLocationsToInsertInOverflow.size() && bc.tableLocationsToInsertInOverflow.get(ii) == tableLocation) { - bc.overflowLocationForEqualityCheck.add(allocatedOverflowLocation); - bc.chunkPositionsToCheckForEquality.add(chunkPosition); - bc.chunkPositionsToCheckForEquality.add(bc.chunkPositionsToInsertInOverflow.get(ii)); - } - } - - // now we need to do the equality check; so that we can mark things appropriately - int remainingInserts = 0; - - checkPairEquality(bc.chunkPositionsToCheckForEquality, sourceKeyChunks, bc.equalValues); - for (int ii = 0; ii < bc.equalValues.size(); ii++) { - final int chunkPosition = bc.chunkPositionsToCheckForEquality.get(ii * 2 + 1); - final long tableLocation = bc.tableLocationsChunk.get(chunkPosition); - - if (bc.equalValues.get(ii)) { - final long insertedOverflowLocation = bc.overflowLocationForEqualityCheck.get(ii); - // region build overflow duplicate - trackingCallback.invoke(overflowCookieSource.getLong(insertedOverflowLocation), overflowLocationToHashLocation((int) insertedOverflowLocation), sourceChunkIndexKeys.get(chunkPosition)); - // endregion build overflow duplicate - } else { - // we need to try this element again in the next round - bc.chunkPositionsToInsertInOverflow.set(remainingInserts, chunkPosition); - bc.tableLocationsToInsertInOverflow.set(remainingInserts++, tableLocation); - } - } - - bc.chunkPositionsToInsertInOverflow.setSize(remainingInserts); - bc.tableLocationsToInsertInOverflow.setSize(remainingInserts); - } - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - // mixin rehash - // region post-build rehash - doRehash(bc, moveCallback); - // endregion post-build rehash - // endmixin rehash - } - - // region copy hash slots - // endregion copy hash slots - hashSlotOffset += chunkOk.size(); - } - // region post build loop - sourceChunkIndexKeys.close(); - // endregion post build loop - } - } - - // mixin rehash - public void doRehash(BuildContext bc - // region extra rehash arguments - , StateMoveCallback moveCallback - // endregion extra rehash arguments - ) { - long firstBackingChunkLocation; - long lastBackingChunkLocation;// mixin rehash - // region rehash start - // endregion rehash start - while (rehashRequired()) { - // region rehash loop start - // endregion rehash loop start - if (tableHashPivot == tableSize) { - tableSize *= 2; - ensureCapacity(tableSize); - // region rehash ensure capacity - // endregion rehash ensure capacity - } - - final long targetBuckets = Math.min(MAX_TABLE_SIZE, (long)(numEntries / targetLoadFactor)); - final int bucketsToAdd = Math.max(1, (int)Math.min(Math.min(targetBuckets, tableSize) - tableHashPivot, bc.chunkSize)); - - initializeRehashLocations(bc.rehashLocations, bucketsToAdd); - - // fill the overflow bucket locations - overflowLocationSource.fillChunk(bc.overflowFillContext, bc.overflowLocations, RowSequenceFactory.wrapRowKeysChunkAsRowSequence(LongChunk.downcast(bc.rehashLocations))); - // null out the overflow locations in the table - setOverflowLocationsToNull(tableHashPivot - (tableSize >> 1), bucketsToAdd); - - while (bc.overflowLocations.size() > 0) { - // figure out which table location each overflow location maps to - compactOverflowLocations(bc.overflowLocations, bc.overflowLocationsToFetch); - if (bc.overflowLocationsToFetch.size() == 0) { - break; - } - - fillOverflowKeys(bc.overflowContexts, bc.workingKeyChunks, bc.overflowLocationsToFetch); - hashKeyChunks(bc.hashChunk, bc.workingKeyChunks); - convertHashToTableLocations(bc.hashChunk, bc.tableLocationsChunk, tableHashPivot + bucketsToAdd); - - // read the next chunk of overflow locations, which we will be overwriting in the next step - overflowOverflowLocationSource.fillChunkUnordered(bc.overflowOverflowFillContext, bc.overflowLocations, bc.overflowLocationsToFetch); - - // swap the table's overflow pointer with our location - swapOverflowPointers(bc.tableLocationsChunk, bc.overflowLocationsToFetch); - } - - // now rehash the main entries - - rowSetSource.fillChunkUnordered(bc.stateSourceFillContext, bc.workingStateEntries, bc.rehashLocations); - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(bc.workingStateEntries, EMPTY_VALUE, bc.shouldMoveBucket); - - // crush down things that don't exist - LongCompactKernel.compact(bc.rehashLocations, bc.shouldMoveBucket); - - // get the keys from the table - fillKeys(bc.workingFillContexts, bc.workingKeyChunks, bc.rehashLocations); - hashKeyChunks(bc.hashChunk, bc.workingKeyChunks); - convertHashToTableLocations(bc.hashChunk, bc.tableLocationsChunk, tableHashPivot + bucketsToAdd); - - // figure out which ones must move - LongChunkEquals.notEqual(bc.tableLocationsChunk, bc.rehashLocations, bc.shouldMoveBucket); - - firstBackingChunkLocation = -1; - lastBackingChunkLocation = -1; - // flushWriteThrough will have zero-ed out the sourcePositions and destinationLocationPositionInWriteThrough size - - int moves = 0; - for (int ii = 0; ii < bc.shouldMoveBucket.size(); ++ii) { - if (bc.shouldMoveBucket.get(ii)) { - moves++; - final long newHashLocation = bc.tableLocationsChunk.get(ii); - final long oldHashLocation = bc.rehashLocations.get(ii); - - if (newHashLocation > lastBackingChunkLocation) { - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, newHashLocation, keySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - } - - // @StateValueType@ from \QTrackingWritableRowSet\E - final TrackingWritableRowSet stateValueToMove = rowSetSource.getUnsafe(oldHashLocation); - rowSetSource.set(newHashLocation, stateValueToMove); - rowSetSource.set(oldHashLocation, EMPTY_VALUE); - // region rehash move values - final long cookie = cookieSource.getLong(oldHashLocation); - moveCallback.invoke(cookie, (int) newHashLocation); - cookieSource.set(newHashLocation, cookie); - // endregion rehash move values - - bc.sourcePositions.add(ii); - bc.destinationLocationPositionInWriteThrough.add((int)(newHashLocation - firstBackingChunkLocation)); - } - } - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - - // everything has been rehashed now, but we have some table locations that might have an overflow, - // without actually having a main entry. We walk through the empty main entries, pulling non-empty - // overflow locations into the main table - - // figure out which of the two possible locations is empty, because (1) we moved something from it - // or (2) we did not move something to it - bc.overflowLocationsToFetch.setSize(bc.shouldMoveBucket.size()); - final int totalPromotionsToProcess = bc.shouldMoveBucket.size(); - createOverflowPartitions(bc.overflowLocationsToFetch, bc.rehashLocations, bc.shouldMoveBucket, moves); - - for (int loop = 0; loop < 2; loop++) { - final boolean firstLoop = loop == 0; - - if (firstLoop) { - bc.overflowLocationForPromotionLoop.resetFromTypedChunk(bc.overflowLocationsToFetch, 0, moves); - } else { - bc.overflowLocationForPromotionLoop.resetFromTypedChunk(bc.overflowLocationsToFetch, moves, totalPromotionsToProcess - moves); - } - - overflowLocationSource.fillChunk(bc.overflowFillContext, bc.overflowLocations, RowSequenceFactory.wrapRowKeysChunkAsRowSequence(bc.overflowLocationForPromotionLoop)); - IntChunkEquals.notEqual(bc.overflowLocations, QueryConstants.NULL_INT, bc.shouldMoveBucket); - - // crunch the chunk down to relevant locations - LongCompactKernel.compact(bc.overflowLocationForPromotionLoop, bc.shouldMoveBucket); - IntCompactKernel.compact(bc.overflowLocations, bc.shouldMoveBucket); - - IntToLongCast.castInto(IntChunk.downcast(bc.overflowLocations), bc.overflowLocationsAsKeyIndices); - - // now fetch the overflow key values - fillOverflowKeys(bc.overflowContexts, bc.workingKeyChunks, bc.overflowLocationsAsKeyIndices); - // and their state values - overflowRowSetSource.fillChunkUnordered(bc.overflowStateSourceFillContext, bc.workingStateEntries, bc.overflowLocationsAsKeyIndices); - // and where their next pointer is - overflowOverflowLocationSource.fillChunkUnordered(bc.overflowOverflowFillContext, bc.overflowLocationsToMigrate, bc.overflowLocationsAsKeyIndices); - - // we'll have two sorted regions intermingled in the overflowLocationsToFetch, one of them is before the pivot, the other is after the pivot - // so that we can use our write through chunks, we first process the things before the pivot; then have a separate loop for those - // that go after - firstBackingChunkLocation = -1; - lastBackingChunkLocation = -1; - - for (int ii = 0; ii < bc.overflowLocationForPromotionLoop.size(); ++ii) { - final long tableLocation = bc.overflowLocationForPromotionLoop.get(ii); - if ((firstLoop && tableLocation < tableHashPivot) || (!firstLoop && tableLocation >= tableHashPivot)) { - if (tableLocation > lastBackingChunkLocation) { - if (bc.sourcePositions.size() > 0) { - // the permutes here are flushing the write through for the state and overflow locations - - IntPermuteKernel.permute(bc.sourcePositions, bc.overflowLocationsToMigrate, bc.destinationLocationPositionInWriteThrough, bc.writeThroughOverflowLocations); - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - } - - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, tableLocation, keySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - updateWriteThroughOverflow(bc.writeThroughOverflowLocations, firstBackingChunkLocation, lastBackingChunkLocation); - } - bc.sourcePositions.add(ii); - bc.destinationLocationPositionInWriteThrough.add((int)(tableLocation - firstBackingChunkLocation)); - // region promotion move - final long overflowLocation = bc.overflowLocationsAsKeyIndices.get(ii); - rowSetSource.set(tableLocation, overflowRowSetSource.get(overflowLocation)); - overflowRowSetSource.set(overflowLocation, EMPTY_VALUE); - final long cookie = overflowCookieSource.getLong(overflowLocation); - moveCallback.invoke(cookie, (int) tableLocation); - cookieSource.set(tableLocation, cookie); - // endregion promotion move - } - } - - // the permutes are completing the state and overflow promotions write through - IntPermuteKernel.permute(bc.sourcePositions, bc.overflowLocationsToMigrate, bc.destinationLocationPositionInWriteThrough, bc.writeThroughOverflowLocations); - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - - // now mark these overflow locations as free, so that we can reuse them - freeOverflowLocations.ensureCapacity(freeOverflowCount + bc.overflowLocations.size()); - // by sorting them, they will be more likely to be in the same write through chunk when we pull them from the free list - bc.overflowLocations.sort(); - for (int ii = 0; ii < bc.overflowLocations.size(); ++ii) { - freeOverflowLocations.set(freeOverflowCount++, bc.overflowLocations.get(ii)); - } - nullOverflowObjectSources(bc.overflowLocations); - } - - tableHashPivot += bucketsToAdd; - // region rehash loop end - // endregion rehash loop end - } - // region rehash final - // endregion rehash final - } - - public boolean rehashRequired() { - return numEntries > (tableHashPivot * maximumLoadFactor) && tableHashPivot < MAX_TABLE_SIZE; - } - - /** - * This function can be stuck in for debugging if you are breaking the table to make sure each slot still corresponds - * to the correct location. - */ - @SuppressWarnings({"unused", "unchecked"}) - private void verifyKeyHashes() { - final int maxSize = tableHashPivot; - - final ChunkSource.FillContext [] keyFillContext = makeFillContexts(keySources, SharedContext.makeSharedContext(), maxSize); - final WritableChunk [] keyChunks = getWritableKeyChunks(maxSize); - - try (final WritableLongChunk positions = WritableLongChunk.makeWritableChunk(maxSize); - final WritableBooleanChunk exists = WritableBooleanChunk.makeWritableChunk(maxSize); - final WritableIntChunk hashChunk = WritableIntChunk.makeWritableChunk(maxSize); - final WritableLongChunk tableLocationsChunk = WritableLongChunk.makeWritableChunk(maxSize); - final SafeCloseableArray ignored = new SafeCloseableArray<>(keyFillContext); - final SafeCloseableArray ignored2 = new SafeCloseableArray<>(keyChunks); - // @StateChunkName@ from \QObjectChunk\E - final WritableObjectChunk stateChunk = WritableObjectChunk.makeWritableChunk(maxSize); - final ChunkSource.FillContext fillContext = rowSetSource.makeFillContext(maxSize)) { - - rowSetSource.fillChunk(fillContext, stateChunk, RowSetFactory.flat(tableHashPivot)); - - ChunkUtils.fillInOrder(positions); - - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(stateChunk, EMPTY_VALUE, exists); - - // crush down things that don't exist - LongCompactKernel.compact(positions, exists); - - // get the keys from the table - fillKeys(keyFillContext, keyChunks, positions); - hashKeyChunks(hashChunk, keyChunks); - convertHashToTableLocations(hashChunk, tableLocationsChunk, tableHashPivot); - - for (int ii = 0; ii < positions.size(); ++ii) { - if (tableLocationsChunk.get(ii) != positions.get(ii)) { - throw new IllegalStateException(); - } - } - } - } - - void setTargetLoadFactor(final double targetLoadFactor) { - this.targetLoadFactor = targetLoadFactor; - } - - void setMaximumLoadFactor(final double maximumLoadFactor) { - this.maximumLoadFactor = maximumLoadFactor; - } - - private void createOverflowPartitions(WritableLongChunk overflowLocationsToFetch, WritableLongChunk rehashLocations, WritableBooleanChunk shouldMoveBucket, int moves) { - int startWritePosition = 0; - int endWritePosition = moves; - for (int ii = 0; ii < shouldMoveBucket.size(); ++ii) { - if (shouldMoveBucket.get(ii)) { - final long oldHashLocation = rehashLocations.get(ii); - // this needs to be promoted, because we moved it - overflowLocationsToFetch.set(startWritePosition++, oldHashLocation); - } else { - // we didn't move anything into the destination slot; so we need to promote its overflow - final long newEmptyHashLocation = rehashLocations.get(ii) + (tableSize >> 1); - overflowLocationsToFetch.set(endWritePosition++, newEmptyHashLocation); - } - } - } - - private void setOverflowLocationsToNull(long start, int count) { - for (int ii = 0; ii < count; ++ii) { - overflowLocationSource.set(start + ii, QueryConstants.NULL_INT); - } - } - - private void initializeRehashLocations(WritableLongChunk rehashLocations, int bucketsToAdd) { - rehashLocations.setSize(bucketsToAdd); - for (int ii = 0; ii < bucketsToAdd; ++ii) { - rehashLocations.set(ii, tableHashPivot + ii - (tableSize >> 1)); - } - } - - private void compactOverflowLocations(IntChunk overflowLocations, WritableLongChunk overflowLocationsToFetch) { - overflowLocationsToFetch.setSize(0); - for (int ii = 0; ii < overflowLocations.size(); ++ii) { - final int overflowLocation = overflowLocations.get(ii); - if (overflowLocation != QueryConstants.NULL_INT) { - overflowLocationsToFetch.add(overflowLocation); - } - } - } - - private void swapOverflowPointers(LongChunk tableLocationsChunk, LongChunk overflowLocationsToFetch) { - for (int ii = 0; ii < overflowLocationsToFetch.size(); ++ii) { - final long newLocation = tableLocationsChunk.get(ii); - final int existingOverflow = overflowLocationSource.getUnsafe(newLocation); - final long overflowLocation = overflowLocationsToFetch.get(ii); - overflowOverflowLocationSource.set(overflowLocation, existingOverflow); - overflowLocationSource.set(newLocation, (int)overflowLocation); - } - } - - - private void updateWriteThroughOverflow(ResettableWritableIntChunk writeThroughOverflow, long firstPosition, long expectedLastPosition) { - final long firstBackingChunkPosition = overflowLocationSource.resetWritableChunkToBackingStore(writeThroughOverflow, firstPosition); - if (firstBackingChunkPosition != firstPosition) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - if (firstBackingChunkPosition + writeThroughOverflow.size() - 1 != expectedLastPosition) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - } - - // endmixin rehash - - private int allocateOverflowLocation() { - // mixin rehash - if (freeOverflowCount > 0) { - return freeOverflowLocations.getUnsafe(--freeOverflowCount); - } - // endmixin rehash - return nextOverflowLocation++; - } - - private static long updateWriteThroughChunks(ResettableWritableChunk[] writeThroughChunks, long currentHashLocation, ArrayBackedColumnSource[] sources) { - final long firstBackingChunkPosition = sources[0].resetWritableChunkToBackingStore(writeThroughChunks[0], currentHashLocation); - for (int jj = 1; jj < sources.length; ++jj) { - if (sources[jj].resetWritableChunkToBackingStore(writeThroughChunks[jj], currentHashLocation) != firstBackingChunkPosition) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - if (writeThroughChunks[jj].size() != writeThroughChunks[0].size()) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - } - return firstBackingChunkPosition; - } - - private void flushWriteThrough(WritableIntChunk sourcePositions, Chunk[] sourceKeyChunks, WritableIntChunk destinationLocationPositionInWriteThrough, WritableChunk[] writeThroughChunks) { - if (sourcePositions.size() < 0) { - return; - } - for (int jj = 0; jj < keySources.length; ++jj) { - chunkCopiers[jj].permute(sourcePositions, sourceKeyChunks[jj], destinationLocationPositionInWriteThrough, writeThroughChunks[jj]); - } - sourcePositions.setSize(0); - destinationLocationPositionInWriteThrough.setSize(0); - } - - // mixin rehash - private void nullOverflowObjectSources(IntChunk locationsToNull) { - for (ObjectArraySource objectArraySource : overflowKeyColumnsToNull) { - for (int ii = 0; ii < locationsToNull.size(); ++ii) { - objectArraySource.set(locationsToNull.get(ii), null); - } - } - // region nullOverflowObjectSources - for (int ii = 0; ii < locationsToNull.size(); ++ii) { - overflowRowSetSource.set(locationsToNull.get(ii), EMPTY_VALUE); - } - // endregion nullOverflowObjectSources - } - // endmixin rehash - - private void checkKeyEquality(WritableBooleanChunk equalValues, WritableChunk[] workingKeyChunks, Chunk[] sourceKeyChunks) { - for (int ii = 0; ii < sourceKeyChunks.length; ++ii) { - chunkEquals[ii].andEqual(workingKeyChunks[ii], sourceKeyChunks[ii], equalValues); - } - } - - private void checkLhsPermutedEquality(WritableIntChunk chunkPositionsForFetches, Chunk[] sourceKeyChunks, WritableChunk[] overflowKeyChunks, WritableBooleanChunk equalValues) { - chunkEquals[0].equalLhsPermuted(chunkPositionsForFetches, sourceKeyChunks[0], overflowKeyChunks[0], equalValues); - for (int ii = 1; ii < overflowKeySources.length; ++ii) { - chunkEquals[ii].andEqualLhsPermuted(chunkPositionsForFetches, sourceKeyChunks[ii], overflowKeyChunks[ii], equalValues); - } - } - - private void checkPairEquality(WritableIntChunk chunkPositionsToCheckForEquality, Chunk[] sourceKeyChunks, WritableBooleanChunk equalPairs) { - chunkEquals[0].equalPairs(chunkPositionsToCheckForEquality, sourceKeyChunks[0], equalPairs); - for (int ii = 1; ii < keyColumnCount; ++ii) { - chunkEquals[ii].andEqualPairs(chunkPositionsToCheckForEquality, sourceKeyChunks[ii], equalPairs); - } - } - - private void fillKeys(ColumnSource.FillContext[] fillContexts, WritableChunk[] keyChunks, WritableLongChunk tableLocationsChunk) { - fillKeys(keySources, fillContexts, keyChunks, tableLocationsChunk); - } - - private void fillOverflowKeys(ColumnSource.FillContext[] fillContexts, WritableChunk[] keyChunks, WritableLongChunk overflowLocationsChunk) { - fillKeys(overflowKeySources, fillContexts, keyChunks, overflowLocationsChunk); - } - - private static void fillKeys(ArrayBackedColumnSource[] keySources, ColumnSource.FillContext[] fillContexts, WritableChunk[] keyChunks, WritableLongChunk keyIndices) { - for (int ii = 0; ii < keySources.length; ++ii) { - keySources[ii].fillChunkUnordered(fillContexts[ii], keyChunks[ii], keyIndices); - } - } - - private void hashKeyChunks(WritableIntChunk hashChunk, Chunk[] sourceKeyChunks) { - chunkHashers[0].hashInitial(sourceKeyChunks[0], hashChunk); - for (int ii = 1; ii < sourceKeyChunks.length; ++ii) { - chunkHashers[ii].hashUpdate(sourceKeyChunks[ii], hashChunk); - } - } - - private void getKeyChunks(ColumnSource[] sources, ColumnSource.GetContext[] contexts, Chunk[] chunks, RowSequence rowSequence) { - for (int ii = 0; ii < chunks.length; ++ii) { - chunks[ii] = sources[ii].getChunk(contexts[ii], rowSequence); - } - } - - // mixin prev - private void getPrevKeyChunks(ColumnSource[] sources, ColumnSource.GetContext[] contexts, Chunk[] chunks, RowSequence rowSequence) { - for (int ii = 0; ii < chunks.length; ++ii) { - chunks[ii] = sources[ii].getPrevChunk(contexts[ii], rowSequence); - } - } - // endmixin prev - - // region probe wrappers - void processRemoves(@NotNull final ColumnSource[] keySources, @NotNull final RowSet removedRowSet, @NotNull final IncrementalByAggregationUpdateTracker aggregationUpdateTracker) { - if (removedRowSet.isEmpty()) { - return; - } - try (final ProbeContext pc = makeProbeContext(keySources, removedRowSet.size())) { - decorationProbe(pc, removedRowSet, keySources, true, aggregationUpdateTracker::processRemove); - } - } - - void processShift(@NotNull final ColumnSource[] keySources, @NotNull final RowSet preShiftedRowSet, @NotNull final IncrementalByAggregationUpdateTracker aggregationUpdateTracker) { - if (preShiftedRowSet.isEmpty()) { - return; - } - try (final ProbeContext pc = makeProbeContext(keySources, preShiftedRowSet.size())) { - decorationProbe(pc, preShiftedRowSet, keySources, true, aggregationUpdateTracker::processShift); - } - } - - void processModifies(@NotNull final ColumnSource[] keySources, @NotNull final RowSet modifiedRowSet, @NotNull final IncrementalByAggregationUpdateTracker aggregationUpdateTracker) { - if (modifiedRowSet.isEmpty()) { - return; - } - try (final ProbeContext pc = makeProbeContext(keySources, modifiedRowSet.size())) { - decorationProbe(pc, modifiedRowSet, keySources, false, aggregationUpdateTracker::processModify); - } - } - // endregion probe wrappers - - // mixin decorationProbe - class ProbeContext implements Context { - final int chunkSize; - - final ColumnSource.FillContext stateSourceFillContext; - final ColumnSource.FillContext overflowFillContext; - final ColumnSource.FillContext overflowOverflowFillContext; - - final SharedContext sharedFillContext; - final ColumnSource.FillContext[] workingFillContexts; - final SharedContext sharedOverflowContext; - final ColumnSource.FillContext[] overflowContexts; - - // the chunk of hashcodes - final WritableIntChunk hashChunk; - // the chunk of positions within our table - final WritableLongChunk tableLocationsChunk; - - // the chunk of right indices that we read from the hash table, the empty right index is used as a sentinel that the - // state exists; otherwise when building from the left it is always null - // @WritableStateChunkType@ from \QWritableObjectChunk\E - final WritableObjectChunk workingStateEntries; - - // the overflow locations that we need to get from the overflowLocationSource (or overflowOverflowLocationSource) - final WritableLongChunk overflowLocationsToFetch; - // the overflow position in the working keychunks, parallel to the overflowLocationsToFetch - final WritableIntChunk overflowPositionInWorkingChunk; - // values we have read from the overflow locations sources - final WritableIntChunk overflowLocations; - // when fetching from the overflow, we record which chunk position we are fetching for - final WritableIntChunk chunkPositionsForFetches; - - final WritableBooleanChunk equalValues; - final WritableChunk[] workingKeyChunks; - - final SharedContext sharedProbeContext; - // the contexts for filling from our key columns - final ChunkSource.GetContext[] probeContexts; - - // region probe context - // the chunk of indices created from our RowSequence, used to write into the hash table - final WritableLongChunk keyIndices; - // endregion probe context - final boolean haveSharedContexts; - - private ProbeContext(ColumnSource[] probeSources, - int chunkSize - // region probe context constructor args - // endregion probe context constructor args - ) { - Assert.gtZero(chunkSize, "chunkSize"); - this.chunkSize = chunkSize; - haveSharedContexts = probeSources.length > 1; - if (haveSharedContexts) { - sharedFillContext = SharedContext.makeSharedContext(); - sharedOverflowContext = SharedContext.makeSharedContext(); - sharedProbeContext = SharedContext.makeSharedContext(); - } else { - // No point in the additional work implied by these being non null. - sharedFillContext = null; - sharedOverflowContext = null; - sharedProbeContext = null; - } - workingFillContexts = makeFillContexts(keySources, sharedFillContext, chunkSize); - overflowContexts = makeFillContexts(overflowKeySources, sharedOverflowContext, chunkSize); - probeContexts = makeGetContexts(probeSources, sharedProbeContext, chunkSize); - // region probe context constructor - keyIndices = WritableLongChunk.makeWritableChunk(CHUNK_SIZE); - // endregion probe context constructor - stateSourceFillContext = rowSetSource.makeFillContext(chunkSize); - overflowFillContext = overflowLocationSource.makeFillContext(chunkSize); - overflowOverflowFillContext = overflowOverflowLocationSource.makeFillContext(chunkSize); - hashChunk = WritableIntChunk.makeWritableChunk(chunkSize); - tableLocationsChunk = WritableLongChunk.makeWritableChunk(chunkSize); - // @WritableStateChunkName@ from \QWritableObjectChunk\E - workingStateEntries = WritableObjectChunk.makeWritableChunk(chunkSize); - overflowLocationsToFetch = WritableLongChunk.makeWritableChunk(chunkSize); - overflowPositionInWorkingChunk = WritableIntChunk.makeWritableChunk(chunkSize); - overflowLocations = WritableIntChunk.makeWritableChunk(chunkSize); - chunkPositionsForFetches = WritableIntChunk.makeWritableChunk(chunkSize); - equalValues = WritableBooleanChunk.makeWritableChunk(chunkSize); - workingKeyChunks = getWritableKeyChunks(chunkSize); - } - - private void resetSharedContexts() { - if (!haveSharedContexts) { - return; - } - sharedFillContext.reset(); - sharedOverflowContext.reset(); - sharedProbeContext.reset(); - } - - private void closeSharedContexts() { - if (!haveSharedContexts) { - return; - } - sharedFillContext.close(); - sharedOverflowContext.close(); - sharedProbeContext.close(); - } - - @Override - public void close() { - stateSourceFillContext.close(); - overflowFillContext.close(); - overflowOverflowFillContext.close(); - closeArray(workingFillContexts); - closeArray(overflowContexts); - closeArray(probeContexts); - hashChunk.close(); - tableLocationsChunk.close(); - workingStateEntries.close(); - overflowLocationsToFetch.close(); - overflowPositionInWorkingChunk.close(); - overflowLocations.close(); - chunkPositionsForFetches.close(); - equalValues.close(); - closeArray(workingKeyChunks); - closeSharedContexts(); - // region probe context close - keyIndices.close(); - // endregion probe context close - closeSharedContexts(); - } - } - - ProbeContext makeProbeContext(ColumnSource[] probeSources, - long maxSize - // region makeProbeContext args - // endregion makeProbeContext args - ) { - return new ProbeContext(probeSources, (int)Math.min(maxSize, CHUNK_SIZE) - // region makeProbeContext arg pass - // endregion makeProbeContext arg pass - ); - } - - private void decorationProbe(ProbeContext pc - , RowSequence probeIndex - , final ColumnSource[] probeSources - // mixin prev - , boolean usePrev - // endmixin prev - // region additional probe arguments - , final StateTrackingCallback trackingCallback - // endregion additional probe arguments - ) { - // region probe start - // endregion probe start - long hashSlotOffset = 0; - - try (final RowSequence.Iterator rsIt = probeIndex.getRowSequenceIterator(); - // region probe additional try resources - // endregion probe additional try resources - ) { - //noinspection unchecked - final Chunk [] sourceKeyChunks = new Chunk[keyColumnCount]; - - // region probe initialization - // endregion probe initialization - - while (rsIt.hasMore()) { - // we reset shared contexts early to avoid carrying around state that can't be reused. - pc.resetSharedContexts(); - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(pc.chunkSize); - final int chunkSize = chunkOk.intSize(); - - // region probe loop initialization - chunkOk.fillRowKeyChunk(pc.keyIndices); - // endregion probe loop initialization - - // get our keys, hash them, and convert them to table locations - // mixin prev - if (usePrev) { - getPrevKeyChunks(probeSources, pc.probeContexts, sourceKeyChunks, chunkOk); - } else { - // endmixin prev - getKeyChunks(probeSources, pc.probeContexts, sourceKeyChunks, chunkOk); - // mixin prev - } - // endmixin prev - hashKeyChunks(pc.hashChunk, sourceKeyChunks); - convertHashToTableLocations(pc.hashChunk, pc.tableLocationsChunk); - - // get the keys from the table - fillKeys(pc.workingFillContexts, pc.workingKeyChunks, pc.tableLocationsChunk); - - // and the corresponding states - // - if a value is empty; we don't care about it - // - otherwise we check for equality; if we are equal, we have found our thing to set - // (or to complain if we are already set) - // - if we are not equal, then we are an overflow block - rowSetSource.fillChunkUnordered(pc.stateSourceFillContext, pc.workingStateEntries, pc.tableLocationsChunk); - - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(pc.workingStateEntries, EMPTY_VALUE, pc.equalValues); - checkKeyEquality(pc.equalValues, pc.workingKeyChunks, sourceKeyChunks); - - pc.overflowPositionInWorkingChunk.setSize(0); - pc.overflowLocationsToFetch.setSize(0); - - for (int ii = 0; ii < pc.equalValues.size(); ++ii) { - if (pc.equalValues.get(ii)) { - // region probe main found - final long tableLocation = pc.tableLocationsChunk.get(ii); - cookieSource.set(tableLocation, trackingCallback.invoke(cookieSource.getLong(tableLocation), (int) tableLocation, pc.keyIndices.get(ii))); - // endregion probe main found - } else if (pc.workingStateEntries.get(ii) != EMPTY_VALUE) { - // we must handle this as part of the overflow bucket - pc.overflowPositionInWorkingChunk.add(ii); - pc.overflowLocationsToFetch.add(pc.tableLocationsChunk.get(ii)); - } else { - // region probe main not found - // endregion probe main not found - } - } - - overflowLocationSource.fillChunkUnordered(pc.overflowFillContext, pc.overflowLocations, pc.overflowLocationsToFetch); - - while (pc.overflowLocationsToFetch.size() > 0) { - pc.overflowLocationsToFetch.setSize(0); - pc.chunkPositionsForFetches.setSize(0); - for (int ii = 0; ii < pc.overflowLocations.size(); ++ii) { - final int overflowLocation = pc.overflowLocations.get(ii); - final int chunkPosition = pc.overflowPositionInWorkingChunk.get(ii); - - // if the overflow slot is null, this state is not responsive to the join so we can ignore it - if (overflowLocation != QueryConstants.NULL_INT) { - pc.overflowLocationsToFetch.add(overflowLocation); - pc.chunkPositionsForFetches.add(chunkPosition); - } else { - // region probe overflow not found - // endregion probe overflow not found - } - } - - // if the slot is non-null, then we need to fetch the overflow values for comparison - fillOverflowKeys(pc.overflowContexts, pc.workingKeyChunks, pc.overflowLocationsToFetch); - - // region probe overflow state source fill - // endregion probe overflow state source fill - - // now compare the value in our workingKeyChunks to the value in the sourceChunk - checkLhsPermutedEquality(pc.chunkPositionsForFetches, sourceKeyChunks, pc.workingKeyChunks, pc.equalValues); - - // we write back into the overflowLocationsToFetch, so we can't set its size to zero. Instead - // we overwrite the elements in the front of the chunk referenced by a position cursor - int overflowPosition = 0; - for (int ii = 0; ii < pc.equalValues.size(); ++ii) { - final long overflowLocation = pc.overflowLocationsToFetch.get(ii); - final int chunkPosition = pc.chunkPositionsForFetches.get(ii); - - if (pc.equalValues.get(ii)) { - // region probe overflow found - overflowCookieSource.set(overflowLocation, trackingCallback.invoke(overflowCookieSource.getLong(overflowLocation), overflowLocationToHashLocation((int) overflowLocation), pc.keyIndices.get(chunkPosition))); - // endregion probe overflow found - } else { - // otherwise, we need to repeat the overflow calculation, with our next overflow fetch - pc.overflowLocationsToFetch.set(overflowPosition, overflowLocation); - pc.overflowPositionInWorkingChunk.set(overflowPosition, chunkPosition); - overflowPosition++; - } - } - pc.overflowLocationsToFetch.setSize(overflowPosition); - pc.overflowPositionInWorkingChunk.setSize(overflowPosition); - - overflowOverflowLocationSource.fillChunkUnordered(pc.overflowOverflowFillContext, pc.overflowLocations, pc.overflowLocationsToFetch); - } - - // region probe complete - // endregion probe complete - hashSlotOffset += chunkSize; - } - - // region probe cleanup - // endregion probe cleanup - } - // region probe final - // endregion probe final - } - // endmixin decorationProbe - - private void convertHashToTableLocations(WritableIntChunk hashChunk, WritableLongChunk tablePositionsChunk) { - // mixin rehash - // NOTE that this mixin section is a bit ugly, we are spanning the two functions so that we can avoid using tableHashPivot and having the unused pivotPoint parameter - convertHashToTableLocations(hashChunk, tablePositionsChunk, tableHashPivot); - } - - private void convertHashToTableLocations(WritableIntChunk hashChunk, WritableLongChunk tablePositionsChunk, int pivotPoint) { - // endmixin rehash - - // turn hash codes into indices within our table - for (int ii = 0; ii < hashChunk.size(); ++ii) { - final int hash = hashChunk.get(ii); - // mixin rehash - final int location = hashToTableLocation(pivotPoint, hash); - // endmixin rehash - // altmixin rehash: final int location = hashToTableLocation(hash); - tablePositionsChunk.set(ii, location); - } - tablePositionsChunk.setSize(hashChunk.size()); - } - - private int hashToTableLocation( - // mixin rehash - int pivotPoint, - // endmixin rehash - int hash) { - // altmixin rehash: final \ - int location = hash & (tableSize - 1); - // mixin rehash - if (location >= pivotPoint) { - location -= (tableSize >> 1); - } - // endmixin rehash - return location; - } - - // region extraction functions - ObjectArraySource getRowSetSource() { - return rowSetSource; - } - - ObjectArraySource getOverflowRowSetSource() { - return overflowRowSetSource; - } - - ColumnSource getRowSetHashTableSource() { - //noinspection unchecked - final ColumnSource indexHashTableSource = new HashTableColumnSource(TrackingWritableRowSet.class, rowSetSource, overflowRowSetSource); - indexHashTableSource.startTrackingPrevValues(); - return indexHashTableSource; - } - - void clearCookies() { - for (int si = 0; si < tableSize; ++si) { - cookieSource.set(si, NULL_COOKIE); - } - for (int osi = 0; osi < nextOverflowLocation; ++ osi) { - overflowCookieSource.set(osi, NULL_COOKIE); - } - } - - void applyShift(final int stateSlot, - final long beginRange, final long endRange, final long shiftDelta, - @NotNull final ShiftAppliedCallback shiftAppliedCallback) { - if (isOverflowLocation(stateSlot)) { - final int overflowSlot = hashLocationToOverflowLocation(stateSlot); - final WritableRowSet overflowStateRowSet = overflowRowSetSource.get(overflowSlot); - if (RowSetShiftData.applyShift(overflowStateRowSet, beginRange, endRange, shiftDelta)) { - overflowCookieSource.set(overflowSlot, shiftAppliedCallback.invoke(overflowCookieSource.getLong(overflowSlot), stateSlot)); - } - } else { - final WritableRowSet stateRowSet = rowSetSource.get(stateSlot); - if (RowSetShiftData.applyShift(stateRowSet, beginRange, endRange, shiftDelta)) { - cookieSource.set(stateSlot, shiftAppliedCallback.invoke(cookieSource.getLong(stateSlot), stateSlot)); - } - } - } - - // endregion extraction functions - - @NotNull - private static ColumnSource.FillContext[] makeFillContexts(ColumnSource[] keySources, final SharedContext sharedContext, int chunkSize) { - final ColumnSource.FillContext[] workingFillContexts = new ColumnSource.FillContext[keySources.length]; - for (int ii = 0; ii < keySources.length; ++ii) { - workingFillContexts[ii] = keySources[ii].makeFillContext(chunkSize, sharedContext); - } - return workingFillContexts; - } - - private static ColumnSource.GetContext[] makeGetContexts(ColumnSource [] sources, final SharedContext sharedState, int chunkSize) { - final ColumnSource.GetContext[] contexts = new ColumnSource.GetContext[sources.length]; - for (int ii = 0; ii < sources.length; ++ii) { - contexts[ii] = sources[ii].makeGetContext(chunkSize, sharedState); - } - return contexts; - } - - @NotNull - private WritableChunk[] getWritableKeyChunks(int chunkSize) { - //noinspection unchecked - final WritableChunk[] workingKeyChunks = new WritableChunk[keyChunkTypes.length]; - for (int ii = 0; ii < keyChunkTypes.length; ++ii) { - workingKeyChunks[ii] = keyChunkTypes[ii].makeWritableChunk(chunkSize); - } - return workingKeyChunks; - } - - @NotNull - private ResettableWritableChunk[] getResettableWritableKeyChunks() { - //noinspection unchecked - final ResettableWritableChunk[] workingKeyChunks = new ResettableWritableChunk[keyChunkTypes.length]; - for (int ii = 0; ii < keyChunkTypes.length; ++ii) { - workingKeyChunks[ii] = keyChunkTypes[ii].makeResettableWritableChunk(); - } - return workingKeyChunks; - } - - // region getStateValue - // endregion getStateValue - - // region overflowLocationToHashLocation - static boolean isOverflowLocation(final int hashSlot) { - return HashTableColumnSource.isOverflowLocation(hashSlot); - } - - static int hashLocationToOverflowLocation(final int hashSlot) { - return HashTableColumnSource.hashLocationToOverflowLocation(hashSlot); - } - - private static int overflowLocationToHashLocation(final int overflowSlot) { - return (int)HashTableColumnSource.overflowLocationToHashLocation(overflowSlot); - } - // endregion overflowLocationToHashLocation - - - static int hashTableSize(long initialCapacity) { - return (int)Math.max(MINIMUM_INITIAL_HASH_SIZE, Math.min(MAX_TABLE_SIZE, Long.highestOneBit(initialCapacity) * 2)); - } - -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedOperatorAggregationStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedOperatorAggregationStateManager.java index 7682137a67d..b3565096a6c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedOperatorAggregationStateManager.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedOperatorAggregationStateManager.java @@ -57,7 +57,7 @@ class IncrementalChunkedOperatorAggregationStateManager { // region constants public static final int CHUNK_SIZE = ChunkedOperatorAggregationHelper.CHUNK_SIZE; - private static final int MINIMUM_INITIAL_HASH_SIZE = CHUNK_SIZE; + public static final int MINIMUM_INITIAL_HASH_SIZE = CHUNK_SIZE; private static final long MAX_TABLE_SIZE = HashTableColumnSource.MINIMUM_OVERFLOW_HASH_SLOT; // endregion constants diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.java deleted file mode 100644 index 5030991e933..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.java +++ /dev/null @@ -1,1357 +0,0 @@ -/* - * Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending - */ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.base.verify.Require; -import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.Any; -import io.deephaven.chunk.attributes.ChunkPositions; -import io.deephaven.chunk.attributes.HashCodes; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.*; -import io.deephaven.engine.table.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.engine.rowset.chunkattributes.RowKeys; -import io.deephaven.util.QueryConstants; -import io.deephaven.chunk.util.hashing.*; -// this is ugly to have twice, but we do need it twice for replication -// @StateChunkIdentityName@ from \QObjectChunkIdentity\E -import io.deephaven.chunk.util.hashing.ObjectChunkIdentityEquals; -import io.deephaven.engine.table.impl.sort.permute.PermuteKernel; -import io.deephaven.engine.table.impl.sort.timsort.LongIntTimsortKernel; -import io.deephaven.engine.table.impl.sources.*; -import io.deephaven.engine.table.impl.util.*; - -// mixin rehash -import java.util.Arrays; -import io.deephaven.engine.table.impl.sort.permute.IntPermuteKernel; -// @StateChunkTypeEnum@ from \QObject\E -import io.deephaven.engine.table.impl.sort.permute.ObjectPermuteKernel; -import io.deephaven.engine.table.impl.util.compact.IntCompactKernel; -import io.deephaven.engine.table.impl.util.compact.LongCompactKernel; -// endmixin rehash - -import io.deephaven.util.SafeCloseableArray; -import org.jetbrains.annotations.NotNull; - -// region extra imports -import io.deephaven.engine.table.impl.HashTableAnnotations; -import org.apache.commons.lang3.mutable.MutableInt; -// endregion extra imports - -import static io.deephaven.util.SafeCloseable.closeArray; - -// region class visibility -public -// endregion class visibility -class StaticChunkedByAggregationStateManager - // region extensions - // endregion extensions -{ - // region constants - private static final int CHUNK_SIZE = IncrementalChunkedByAggregationStateManager.CHUNK_SIZE; - private static final int MINIMUM_INITIAL_HASH_SIZE = IncrementalChunkedByAggregationStateManager.MINIMUM_INITIAL_HASH_SIZE; - private static final int MAX_TABLE_SIZE = IncrementalChunkedByAggregationStateManager.MAX_TABLE_SIZE; - // endregion constants - - // mixin rehash - static final double DEFAULT_MAX_LOAD_FACTOR = 0.75; - static final double DEFAULT_TARGET_LOAD_FACTOR = 0.70; - // endmixin rehash - - // region preamble variables - private static final int EXISTING_HASH_SLOT = -1; - // endregion preamble variables - - @HashTableAnnotations.EmptyStateValue - // @NullStateValue@ from \Qnull\E, @StateValueType@ from \QObject\E - private static final Object EMPTY_VALUE = null; - - // mixin getStateValue - // region overflow pivot - private static final int MINIMUM_OVERFLOW_HASH_SLOT = IncrementalChunkedByAggregationStateManager.MINIMUM_OVERFLOW_HASH_SLOT; - // endregion overflow pivot - // endmixin getStateValue - - // the number of slots in our table - // mixin rehash - private int tableSize; - // endmixin rehash - // altmixin rehash: private final int tableSize; - - // how many key columns we have - private final int keyColumnCount; - - // mixin rehash - private long numEntries = 0; - - /** Our table size must be 2^L (i.e. a power of two); and the pivot is between 2^(L-1) and 2^L. - * - *

When hashing a value, if hashCode % 2^L < tableHashPivot; then the destination location is hashCode % 2^L. - * If hashCode % 2^L >= tableHashPivot, then the destination location is hashCode % 2^(L-1). Once the pivot reaches - * the table size, we can simply double the table size and repeat the process.

- * - *

This has the effect of only using hash table locations < hashTablePivot. When we want to expand the table - * we can move some of the entries from the location {@code tableHashPivot - 2^(L-1)} to tableHashPivot. This - * provides for incremental expansion of the hash table, without the need for a full rehash.

- */ - private int tableHashPivot; - - // the table will be rehashed to a load factor of targetLoadFactor if our loadFactor exceeds maximumLoadFactor - // or if it falls below minimum load factor we will instead contract the table - private double targetLoadFactor = DEFAULT_TARGET_LOAD_FACTOR; - private double maximumLoadFactor = DEFAULT_MAX_LOAD_FACTOR; - // TODO: We do not yet support contraction - // private final double minimumLoadFactor = 0.5; - - private final IntegerArraySource freeOverflowLocations = new IntegerArraySource(); - private int freeOverflowCount = 0; - // endmixin rehash - - // the keys for our hash entries - private final ArrayBackedColumnSource[] keySources; - // the location of any overflow entry in this bucket - private final IntegerArraySource overflowLocationSource = new IntegerArraySource(); - - // we are going to also reuse this for our state entry, so that we do not need additional storage - @HashTableAnnotations.StateColumnSource - // @StateColumnSourceType@ from \QObjectArraySource\E - private final ObjectArraySource rowSetSource - // @StateColumnSourceConstructor@ from \QObjectArraySource<>(Object.class)\E - = new ObjectArraySource<>(Object.class); - - // the keys for overflow - private int nextOverflowLocation = 0; - private final ArrayBackedColumnSource [] overflowKeySources; - // the location of the next key in an overflow bucket - private final IntegerArraySource overflowOverflowLocationSource = new IntegerArraySource(); - // the overflow buckets for the state source - @HashTableAnnotations.OverflowStateColumnSource - // @StateColumnSourceType@ from \QObjectArraySource\E - private final ObjectArraySource overflowRowSetSource - // @StateColumnSourceConstructor@ from \QObjectArraySource<>(Object.class)\E - = new ObjectArraySource<>(Object.class); - - // the type of each of our key chunks - private final ChunkType[] keyChunkTypes; - - // the operators for hashing and various equality methods - private final ChunkHasher[] chunkHashers; - private final ChunkEquals[] chunkEquals; - private final PermuteKernel[] chunkCopiers; - - // mixin rehash - // If we have objects in our key columns, then we should null them out if we delete an overflow row, this only - // applies to ObjectArraySources, for primitives we are content to leave the dead entries in the tables, because - // they will not affect GC. - private final ObjectArraySource[] overflowKeyColumnsToNull; - // endmixin rehash - - // region extra variables - private final IntegerArraySource statePositionInInsertedHashSlots = new IntegerArraySource(); - private final IntegerArraySource overflowStatePositionInInsertedHashSlots = new IntegerArraySource(); - // endregion extra variables - - StaticChunkedByAggregationStateManager(ColumnSource[] tableKeySources - , int tableSize - // region constructor arguments - , double targetLoadFactor - , double maximumLoadFactor - // endregion constructor arguments - ) { - // region super - // endregion super - keyColumnCount = tableKeySources.length; - - this.tableSize = tableSize; - Require.leq(tableSize, "tableSize", MAX_TABLE_SIZE); - Require.gtZero(tableSize, "tableSize"); - Require.eq(Integer.bitCount(tableSize), "Integer.bitCount(tableSize)", 1); - // mixin rehash - this.tableHashPivot = tableSize; - // endmixin rehash - - overflowKeySources = new ArrayBackedColumnSource[keyColumnCount]; - keySources = new ArrayBackedColumnSource[keyColumnCount]; - - keyChunkTypes = new ChunkType[keyColumnCount]; - chunkHashers = new ChunkHasher[keyColumnCount]; - chunkEquals = new ChunkEquals[keyColumnCount]; - chunkCopiers = new PermuteKernel[keyColumnCount]; - - for (int ii = 0; ii < keyColumnCount; ++ii) { - // the sources that we will use to store our hash table - keySources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(tableSize, tableKeySources[ii].getType()); - keyChunkTypes[ii] = tableKeySources[ii].getChunkType(); - - overflowKeySources[ii] = ArrayBackedColumnSource.getMemoryColumnSource(CHUNK_SIZE, tableKeySources[ii].getType()); - - chunkHashers[ii] = ChunkHasher.makeHasher(keyChunkTypes[ii]); - chunkEquals[ii] = ChunkEquals.makeEqual(keyChunkTypes[ii]); - chunkCopiers[ii] = PermuteKernel.makePermuteKernel(keyChunkTypes[ii]); - } - - // mixin rehash - overflowKeyColumnsToNull = Arrays.stream(overflowKeySources).filter(x -> x instanceof ObjectArraySource).map(x -> (ObjectArraySource)x).toArray(ObjectArraySource[]::new); - // endmixin rehash - - // region constructor - setTargetLoadFactor(targetLoadFactor); - setMaximumLoadFactor(maximumLoadFactor); - // endregion constructor - - ensureCapacity(tableSize); - } - - private void ensureCapacity(int tableSize) { - rowSetSource.ensureCapacity(tableSize); - overflowLocationSource.ensureCapacity(tableSize); - for (int ii = 0; ii < keyColumnCount; ++ii) { - keySources[ii].ensureCapacity(tableSize); - } - // region ensureCapacity - statePositionInInsertedHashSlots.ensureCapacity(tableSize); - // endregion ensureCapacity - } - - private void ensureOverflowCapacity(WritableIntChunk chunkPositionsToInsertInOverflow) { - final int locationsToAllocate = chunkPositionsToInsertInOverflow.size(); - // mixin rehash - if (freeOverflowCount >= locationsToAllocate) { - return; - } - final int newCapacity = nextOverflowLocation + locationsToAllocate - freeOverflowCount; - // endmixin rehash - // altmixin rehash: final int newCapacity = nextOverflowLocation + locationsToAllocate; - overflowOverflowLocationSource.ensureCapacity(newCapacity); - overflowRowSetSource.ensureCapacity(newCapacity); - //noinspection ForLoopReplaceableByForEach - for (int ii = 0; ii < overflowKeySources.length; ++ii) { - overflowKeySources[ii].ensureCapacity(newCapacity); - } - // region ensureOverflowCapacity - overflowStatePositionInInsertedHashSlots.ensureCapacity(newCapacity); - // endregion ensureOverflowCapacity - } - - // region build wrappers - int buildTable(@NotNull final Table sourceTable, @NotNull final ColumnSource[] keySources, @NotNull final IntegerArraySource insertedHashSlots) { - if (sourceTable.isEmpty()) { - return 0; - } - final MutableInt insertedHashSlotsLength = new MutableInt(0); - try (final BuildContext bc = makeBuildContext(keySources, sourceTable.size())) { - buildTable(bc, sourceTable.getRowSet(), keySources, insertedHashSlots, insertedHashSlotsLength); - } - return insertedHashSlotsLength.intValue(); - } - // endregion build wrappers - - class BuildContext implements Context { - final int chunkSize; - - final LongIntTimsortKernel.LongIntSortKernelContext sortContext; - final ColumnSource.FillContext stateSourceFillContext; - // mixin rehash - final ColumnSource.FillContext overflowStateSourceFillContext; - // endmixin rehash - final ColumnSource.FillContext overflowFillContext; - final ColumnSource.FillContext overflowOverflowFillContext; - - // the chunk of hashcodes - final WritableIntChunk hashChunk; - // the chunk of positions within our table - final WritableLongChunk tableLocationsChunk; - - final ResettableWritableChunk[] writeThroughChunks = getResettableWritableKeyChunks(); - final WritableIntChunk sourcePositions; - final WritableIntChunk destinationLocationPositionInWriteThrough; - - final WritableBooleanChunk filledValues; - final WritableBooleanChunk equalValues; - - // the overflow locations that we need to get from the overflowLocationSource (or overflowOverflowLocationSource) - final WritableLongChunk overflowLocationsToFetch; - // the overflow position in the working key chunks, parallel to the overflowLocationsToFetch - final WritableIntChunk overflowPositionInSourceChunk; - - // the position with our hash table that we should insert a value into - final WritableLongChunk insertTableLocations; - // the position in our chunk, parallel to the workingChunkInsertTablePositions - final WritableIntChunk insertPositionsInSourceChunk; - - // we sometimes need to check two positions within a single chunk for equality, this contains those positions as pairs - final WritableIntChunk chunkPositionsToCheckForEquality; - // While processing overflow insertions, parallel to the chunkPositions to check for equality, the overflow location that - // is represented by the first of the pairs in chunkPositionsToCheckForEquality - final WritableLongChunk overflowLocationForEqualityCheck; - - // the chunk of state values that we read from the hash table - // @WritableStateChunkType@ from \QWritableObjectChunk\E - final WritableObjectChunk workingStateEntries; - - // the chunks for getting key values from the hash table - final WritableChunk[] workingKeyChunks; - final WritableChunk[] overflowKeyChunks; - - // when fetching from the overflow, we record which chunk position we are fetching for - final WritableIntChunk chunkPositionsForFetches; - // which positions in the chunk we are inserting into the overflow - final WritableIntChunk chunkPositionsToInsertInOverflow; - // which table locations we are inserting into the overflow - final WritableLongChunk tableLocationsToInsertInOverflow; - - // values we have read from the overflow locations sources - final WritableIntChunk overflowLocations; - - // mixin rehash - final WritableLongChunk rehashLocations; - final WritableIntChunk overflowLocationsToMigrate; - final WritableLongChunk overflowLocationsAsKeyIndices; - final WritableBooleanChunk shouldMoveBucket; - - final ResettableWritableLongChunk overflowLocationForPromotionLoop = ResettableWritableLongChunk.makeResettableChunk(); - - final ResettableWritableIntChunk writeThroughOverflowLocations = ResettableWritableIntChunk.makeResettableChunk(); - // endmixin rehash - - final SharedContext sharedFillContext; - final ColumnSource.FillContext[] workingFillContexts; - final SharedContext sharedOverflowContext; - final ColumnSource.FillContext[] overflowContexts; - final SharedContext sharedBuildContext; - final ChunkSource.GetContext[] buildContexts; - - // region build context - // endregion build context - - final boolean haveSharedContexts; - - private BuildContext(ColumnSource[] buildSources, - int chunkSize - // region build context constructor args - // endregion build context constructor args - ) { - Assert.gtZero(chunkSize, "chunkSize"); - this.chunkSize = chunkSize; - haveSharedContexts = buildSources.length > 1; - if (haveSharedContexts) { - sharedFillContext = SharedContext.makeSharedContext(); - sharedOverflowContext = SharedContext.makeSharedContext(); - sharedBuildContext = SharedContext.makeSharedContext(); - } else { - // no point in the additional work implied by these not being null. - sharedFillContext = null; - sharedOverflowContext = null; - sharedBuildContext = null; - } - workingFillContexts = makeFillContexts(keySources, sharedFillContext, chunkSize); - overflowContexts = makeFillContexts(overflowKeySources, sharedOverflowContext, chunkSize); - buildContexts = makeGetContexts(buildSources, sharedBuildContext, chunkSize); - // region build context constructor - // endregion build context constructor - sortContext = LongIntTimsortKernel.createContext(chunkSize); - stateSourceFillContext = rowSetSource.makeFillContext(chunkSize); - overflowFillContext = overflowLocationSource.makeFillContext(chunkSize); - overflowOverflowFillContext = overflowOverflowLocationSource.makeFillContext(chunkSize); - hashChunk = WritableIntChunk.makeWritableChunk(chunkSize); - tableLocationsChunk = WritableLongChunk.makeWritableChunk(chunkSize); - sourcePositions = WritableIntChunk.makeWritableChunk(chunkSize); - destinationLocationPositionInWriteThrough = WritableIntChunk.makeWritableChunk(chunkSize); - filledValues = WritableBooleanChunk.makeWritableChunk(chunkSize); - equalValues = WritableBooleanChunk.makeWritableChunk(chunkSize); - overflowLocationsToFetch = WritableLongChunk.makeWritableChunk(chunkSize); - overflowPositionInSourceChunk = WritableIntChunk.makeWritableChunk(chunkSize); - insertTableLocations = WritableLongChunk.makeWritableChunk(chunkSize); - insertPositionsInSourceChunk = WritableIntChunk.makeWritableChunk(chunkSize); - chunkPositionsToCheckForEquality = WritableIntChunk.makeWritableChunk(chunkSize * 2); - overflowLocationForEqualityCheck = WritableLongChunk.makeWritableChunk(chunkSize); - // @WritableStateChunkName@ from \QWritableObjectChunk\E - workingStateEntries = WritableObjectChunk.makeWritableChunk(chunkSize); - workingKeyChunks = getWritableKeyChunks(chunkSize); - overflowKeyChunks = getWritableKeyChunks(chunkSize); - chunkPositionsForFetches = WritableIntChunk.makeWritableChunk(chunkSize); - chunkPositionsToInsertInOverflow = WritableIntChunk.makeWritableChunk(chunkSize); - tableLocationsToInsertInOverflow = WritableLongChunk.makeWritableChunk(chunkSize); - overflowLocations = WritableIntChunk.makeWritableChunk(chunkSize); - // mixin rehash - rehashLocations = WritableLongChunk.makeWritableChunk(chunkSize); - overflowStateSourceFillContext = overflowRowSetSource.makeFillContext(chunkSize); - overflowLocationsToMigrate = WritableIntChunk.makeWritableChunk(chunkSize); - overflowLocationsAsKeyIndices = WritableLongChunk.makeWritableChunk(chunkSize); - shouldMoveBucket = WritableBooleanChunk.makeWritableChunk(chunkSize); - // endmixin rehash - } - - private void resetSharedContexts() { - if (!haveSharedContexts) { - return; - } - sharedFillContext.reset(); - sharedOverflowContext.reset(); - sharedBuildContext.reset(); - } - - private void closeSharedContexts() { - if (!haveSharedContexts) { - return; - } - sharedFillContext.close(); - sharedOverflowContext.close(); - sharedBuildContext.close(); - } - - @Override - public void close() { - sortContext.close(); - stateSourceFillContext.close(); - // mixin rehash - overflowStateSourceFillContext.close(); - // endmixin rehash - overflowFillContext.close(); - overflowOverflowFillContext.close(); - closeArray(workingFillContexts); - closeArray(overflowContexts); - closeArray(buildContexts); - - hashChunk.close(); - tableLocationsChunk.close(); - closeArray(writeThroughChunks); - - sourcePositions.close(); - destinationLocationPositionInWriteThrough.close(); - filledValues.close(); - equalValues.close(); - overflowLocationsToFetch.close(); - overflowPositionInSourceChunk.close(); - insertTableLocations.close(); - insertPositionsInSourceChunk.close(); - chunkPositionsToCheckForEquality.close(); - overflowLocationForEqualityCheck.close(); - workingStateEntries.close(); - closeArray(workingKeyChunks); - closeArray(overflowKeyChunks); - chunkPositionsForFetches.close(); - chunkPositionsToInsertInOverflow.close(); - tableLocationsToInsertInOverflow.close(); - overflowLocations.close(); - // mixin rehash - rehashLocations.close(); - overflowLocationsToMigrate.close(); - overflowLocationsAsKeyIndices.close(); - shouldMoveBucket.close(); - overflowLocationForPromotionLoop.close(); - writeThroughOverflowLocations.close(); - // endmixin rehash - // region build context close - // endregion build context close - closeSharedContexts(); - } - - } - - BuildContext makeBuildContext(ColumnSource[] buildSources, - long maxSize - // region makeBuildContext args - // endregion makeBuildContext args - ) { - return new BuildContext(buildSources, (int)Math.min(CHUNK_SIZE, maxSize) - // region makeBuildContext arg pass - // endregion makeBuildContext arg pass - ); - } - - private void buildTable(final BuildContext bc, - final RowSequence buildIndex, - ColumnSource[] buildSources - // region extra build arguments - , final IntegerArraySource insertedHashSlots - , final MutableInt insertedHashSlotsLength - // endregion extra build arguments - ) { - long hashSlotOffset = 0; - // region build start - // endregion build start - - try (final RowSequence.Iterator rsIt = buildIndex.getRowSequenceIterator(); - // region build initialization try - // endregion build initialization try - ) { - // region build initialization - // RowSet keys extracted from the input RowSet, parallel to the sourceKeyChunks - final WritableLongChunk sourceChunkIndexKeys = WritableLongChunk.makeWritableChunk(bc.chunkSize); - - // Result destination slots recorded during the build, parallel to the sourceKeyChunks and sourceChunkIndexKeys - final WritableIntChunk sourceChunkInsertedHashSlots = WritableIntChunk.makeWritableChunk(bc.chunkSize); - // endregion build initialization - - // chunks to write through to the table key sources - - - //noinspection unchecked - final Chunk [] sourceKeyChunks = new Chunk[buildSources.length]; - - while (rsIt.hasMore()) { - // we reset early to avoid carrying around state for old RowSequence which can't be reused. - bc.resetSharedContexts(); - - final RowSequence chunkOk = rsIt.getNextRowSequenceWithLength(bc.chunkSize); - - getKeyChunks(buildSources, bc.buildContexts, sourceKeyChunks, chunkOk); - hashKeyChunks(bc.hashChunk, sourceKeyChunks); - - // region build loop initialization - chunkOk.fillRowKeyChunk(sourceChunkIndexKeys); - sourceChunkInsertedHashSlots.setSize(bc.hashChunk.size()); - // endregion build loop initialization - - // turn hash codes into indices within our table - convertHashToTableLocations(bc.hashChunk, bc.tableLocationsChunk); - - // now fetch the values from the table, note that we do not order these fetches - fillKeys(bc.workingFillContexts, bc.workingKeyChunks, bc.tableLocationsChunk); - - // and the corresponding states, if a value is null, we've found our insertion point - rowSetSource.fillChunkUnordered(bc.stateSourceFillContext, bc.workingStateEntries, bc.tableLocationsChunk); - - // find things that exist - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(bc.workingStateEntries, EMPTY_VALUE, bc.filledValues); - - // to be equal, the location must exist; and each of the keyChunks must match - bc.equalValues.setSize(bc.filledValues.size()); - bc.equalValues.copyFromChunk(bc.filledValues, 0, 0, bc.filledValues.size()); - checkKeyEquality(bc.equalValues, bc.workingKeyChunks, sourceKeyChunks); - - bc.overflowPositionInSourceChunk.setSize(0); - bc.overflowLocationsToFetch.setSize(0); - bc.insertPositionsInSourceChunk.setSize(0); - bc.insertTableLocations.setSize(0); - - for (int ii = 0; ii < bc.equalValues.size(); ++ii) { - final long tableLocation = bc.tableLocationsChunk.get(ii); - if (bc.equalValues.get(ii)) { - // region build found main - ((RowSetBuilderSequential) rowSetSource.get(tableLocation)).appendKey(sourceChunkIndexKeys.get(ii)); - sourceChunkInsertedHashSlots.set(ii, EXISTING_HASH_SLOT); - // endregion build found main - } else if (bc.filledValues.get(ii)) { - // we must handle this as part of the overflow bucket - bc.overflowPositionInSourceChunk.add(ii); - bc.overflowLocationsToFetch.add(tableLocation); - } else { - // for the values that are empty, we record them in the insert chunks - bc.insertPositionsInSourceChunk.add(ii); - bc.insertTableLocations.add(tableLocation); - } - } - - // we first sort by position; so that we'll not insert things into the table twice or overwrite - // collisions - LongIntTimsortKernel.sort(bc.sortContext, bc.insertPositionsInSourceChunk, bc.insertTableLocations); - - // the first and last valid table location in our writeThroughChunks - long firstBackingChunkLocation = -1; - long lastBackingChunkLocation = -1; - - bc.chunkPositionsToCheckForEquality.setSize(0); - bc.destinationLocationPositionInWriteThrough.setSize(0); - bc.sourcePositions.setSize(0); - - for (int ii = 0; ii < bc.insertPositionsInSourceChunk.size(); ) { - final int firstChunkPositionForHashLocation = bc.insertPositionsInSourceChunk.get(ii); - final long currentHashLocation = bc.insertTableLocations.get(ii); - - // region main insert - final RowSetBuilderSequential mainSlotBuilder = RowSetFactory.builderSequential(); - mainSlotBuilder.appendKey(sourceChunkIndexKeys.get(firstChunkPositionForHashLocation)); - rowSetSource.set(currentHashLocation, mainSlotBuilder); - statePositionInInsertedHashSlots.set(currentHashLocation, chunkPositionToInsertedHashSlotPosition(firstChunkPositionForHashLocation)); - sourceChunkInsertedHashSlots.set(firstChunkPositionForHashLocation, (int) currentHashLocation); - // endregion main insert - // mixin rehash - numEntries++; - // endmixin rehash - - if (currentHashLocation > lastBackingChunkLocation) { - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, currentHashLocation, keySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - } - - bc.sourcePositions.add(firstChunkPositionForHashLocation); - bc.destinationLocationPositionInWriteThrough.add((int)(currentHashLocation - firstBackingChunkLocation)); - - final int currentHashValue = bc.hashChunk.get(firstChunkPositionForHashLocation); - - while (++ii < bc.insertTableLocations.size() && bc.insertTableLocations.get(ii) == currentHashLocation) { - // if this thing is equal to the first one; we should mark the appropriate slot, we don't - // know the types and don't want to make the virtual calls, so we need to just accumulate - // the things to check for equality afterwards - final int chunkPosition = bc.insertPositionsInSourceChunk.get(ii); - if (bc.hashChunk.get(chunkPosition) != currentHashValue) { - // we must be an overflow - bc.overflowPositionInSourceChunk.add(chunkPosition); - bc.overflowLocationsToFetch.add(currentHashLocation); - } else { - // we need to check equality, equal things are the same slot; unequal things are overflow - bc.chunkPositionsToCheckForEquality.add(firstChunkPositionForHashLocation); - bc.chunkPositionsToCheckForEquality.add(chunkPosition); - } - } - } - - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - - checkPairEquality(bc.chunkPositionsToCheckForEquality, sourceKeyChunks, bc.equalValues); - - for (int ii = 0; ii < bc.equalValues.size(); ii++) { - final int chunkPosition = bc.chunkPositionsToCheckForEquality.get(ii * 2 + 1); - final long tableLocation = bc.tableLocationsChunk.get(chunkPosition); - - if (bc.equalValues.get(ii)) { - // region build main duplicate - ((RowSetBuilderSequential) rowSetSource.get(tableLocation)).appendKey(sourceChunkIndexKeys.get(chunkPosition)); - sourceChunkInsertedHashSlots.set(chunkPosition, EXISTING_HASH_SLOT); - // endregion build main duplicate - } else { - // we are an overflow element - bc.overflowPositionInSourceChunk.add(chunkPosition); - bc.overflowLocationsToFetch.add(tableLocation); - } - } - - // now handle overflow - if (bc.overflowPositionInSourceChunk.size() > 0) { - // on the first pass we fill from the table's locations - overflowLocationSource.fillChunkUnordered(bc.overflowFillContext, bc.overflowLocations, bc.overflowLocationsToFetch); - bc.chunkPositionsToInsertInOverflow.setSize(0); - bc.tableLocationsToInsertInOverflow.setSize(0); - - // overflow slots now contains the positions in the overflow columns - - while (bc.overflowPositionInSourceChunk.size() > 0) { - // now we have the overflow slot for each of the things we are interested in. - // if the slot is null, then we can insert it and we are complete. - - bc.overflowLocationsToFetch.setSize(0); - bc.chunkPositionsForFetches.setSize(0); - - // TODO: Crunch it down - for (int ii = 0; ii < bc.overflowLocations.size(); ++ii) { - final int overflowLocation = bc.overflowLocations.get(ii); - final int chunkPosition = bc.overflowPositionInSourceChunk.get(ii); - if (overflowLocation == QueryConstants.NULL_INT) { - // insert me into overflow in the next free overflow slot - bc.chunkPositionsToInsertInOverflow.add(chunkPosition); - bc.tableLocationsToInsertInOverflow.add(bc.tableLocationsChunk.get(chunkPosition)); - } else { - // add to the key positions to fetch - bc.chunkPositionsForFetches.add(chunkPosition); - bc.overflowLocationsToFetch.add(overflowLocation); - } - } - - // if the slot is non-null, then we need to fetch the overflow values for comparison - fillOverflowKeys(bc.overflowContexts, bc.overflowKeyChunks, bc.overflowLocationsToFetch); - - // now compare the value in our overflowKeyChunk to the value in the sourceChunk - checkLhsPermutedEquality(bc.chunkPositionsForFetches, sourceKeyChunks, bc.overflowKeyChunks, bc.equalValues); - - int writePosition = 0; - for (int ii = 0; ii < bc.equalValues.size(); ++ii) { - final int chunkPosition = bc.chunkPositionsForFetches.get(ii); - final long overflowLocation = bc.overflowLocationsToFetch.get(ii); - if (bc.equalValues.get(ii)) { - // region build overflow found - ((RowSetBuilderSequential) overflowRowSetSource.get(overflowLocation)).appendKey(sourceChunkIndexKeys.get(chunkPosition)); - sourceChunkInsertedHashSlots.set(chunkPosition, EXISTING_HASH_SLOT); - // endregion build overflow found - } else { - // otherwise, we need to repeat the overflow calculation, with our next overflow fetch - bc.overflowLocationsToFetch.set(writePosition, overflowLocation); - bc.overflowPositionInSourceChunk.set(writePosition++, chunkPosition); - } - } - bc.overflowLocationsToFetch.setSize(writePosition); - bc.overflowPositionInSourceChunk.setSize(writePosition); - - // on subsequent iterations, we are following the overflow chains, so we fill from the overflowOverflowLocationSource - if (bc.overflowPositionInSourceChunk.size() > 0) { - overflowOverflowLocationSource.fillChunkUnordered(bc.overflowOverflowFillContext, bc.overflowLocations, bc.overflowLocationsToFetch); - } - } - - // make sure we actually have enough room to insert stuff where we would like - ensureOverflowCapacity(bc.chunkPositionsToInsertInOverflow); - - firstBackingChunkLocation = -1; - lastBackingChunkLocation = -1; - bc.destinationLocationPositionInWriteThrough.setSize(0); - bc.sourcePositions.setSize(0); - - // do the overflow insertions, one per table position at a time; until we have no insertions left - while (bc.chunkPositionsToInsertInOverflow.size() > 0) { - // sort by table position - LongIntTimsortKernel.sort(bc.sortContext, bc.chunkPositionsToInsertInOverflow, bc.tableLocationsToInsertInOverflow); - - bc.chunkPositionsToCheckForEquality.setSize(0); - bc.overflowLocationForEqualityCheck.setSize(0); - - for (int ii = 0; ii < bc.chunkPositionsToInsertInOverflow.size(); ) { - final long tableLocation = bc.tableLocationsToInsertInOverflow.get(ii); - final int chunkPosition = bc.chunkPositionsToInsertInOverflow.get(ii); - - final int allocatedOverflowLocation = allocateOverflowLocation(); - - // we are inserting into the head of the list, so we move the existing overflow into our overflow - overflowOverflowLocationSource.set(allocatedOverflowLocation, overflowLocationSource.getUnsafe(tableLocation)); - // and we point the overflow at our slot - overflowLocationSource.set(tableLocation, allocatedOverflowLocation); - - // region build overflow insert - final RowSetBuilderSequential overflowSlotBuilder = RowSetFactory.builderSequential(); - overflowSlotBuilder.appendKey(sourceChunkIndexKeys.get(chunkPosition)); - overflowRowSetSource.set(allocatedOverflowLocation, overflowSlotBuilder); - overflowStatePositionInInsertedHashSlots.set(allocatedOverflowLocation, chunkPositionToInsertedHashSlotPosition(chunkPosition)); - sourceChunkInsertedHashSlots.set(chunkPosition, overflowLocationToHashLocation(allocatedOverflowLocation)); - // endregion build overflow insert - - // mixin rehash - numEntries++; - // endmixin rehash - - // get the backing chunk from the overflow keys - if (allocatedOverflowLocation > lastBackingChunkLocation || allocatedOverflowLocation < firstBackingChunkLocation) { - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, allocatedOverflowLocation, overflowKeySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - } - - // now we must set all of our key values in the overflow - bc.sourcePositions.add(chunkPosition); - bc.destinationLocationPositionInWriteThrough.add((int)(allocatedOverflowLocation - firstBackingChunkLocation)); - - while (++ii < bc.tableLocationsToInsertInOverflow.size() && bc.tableLocationsToInsertInOverflow.get(ii) == tableLocation) { - bc.overflowLocationForEqualityCheck.add(allocatedOverflowLocation); - bc.chunkPositionsToCheckForEquality.add(chunkPosition); - bc.chunkPositionsToCheckForEquality.add(bc.chunkPositionsToInsertInOverflow.get(ii)); - } - } - - // now we need to do the equality check; so that we can mark things appropriately - int remainingInserts = 0; - - checkPairEquality(bc.chunkPositionsToCheckForEquality, sourceKeyChunks, bc.equalValues); - for (int ii = 0; ii < bc.equalValues.size(); ii++) { - final int chunkPosition = bc.chunkPositionsToCheckForEquality.get(ii * 2 + 1); - final long tableLocation = bc.tableLocationsChunk.get(chunkPosition); - - if (bc.equalValues.get(ii)) { - final long insertedOverflowLocation = bc.overflowLocationForEqualityCheck.get(ii); - // region build overflow duplicate - ((RowSetBuilderSequential) overflowRowSetSource.get(insertedOverflowLocation)).appendKey(sourceChunkIndexKeys.get(chunkPosition)); - sourceChunkInsertedHashSlots.set(chunkPosition, EXISTING_HASH_SLOT); - // endregion build overflow duplicate - } else { - // we need to try this element again in the next round - bc.chunkPositionsToInsertInOverflow.set(remainingInserts, chunkPosition); - bc.tableLocationsToInsertInOverflow.set(remainingInserts++, tableLocation); - } - } - - bc.chunkPositionsToInsertInOverflow.setSize(remainingInserts); - bc.tableLocationsToInsertInOverflow.setSize(remainingInserts); - } - flushWriteThrough(bc.sourcePositions, sourceKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - // mixin rehash - // region post-build rehash - doRehash(bc, sourceChunkInsertedHashSlots, insertedHashSlots); - // endregion post-build rehash - // endmixin rehash - } - - // region copy hash slots - // We need to take care of setting our inserted hash slots before rehash can move them - int insertedHashSlotPosition = insertedHashSlotsLength.intValue(); - insertedHashSlots.ensureCapacity(insertedHashSlotPosition + sourceChunkInsertedHashSlots.size()); - for (int ii = 0; ii < sourceChunkInsertedHashSlots.size(); ++ii) { - final int insertedHashSlot = sourceChunkInsertedHashSlots.get(ii); - if (insertedHashSlot != EXISTING_HASH_SLOT) { - insertedHashSlots.set(insertedHashSlotPosition, insertedHashSlot); - if (isOverflowLocation(insertedHashSlot)) { - overflowStatePositionInInsertedHashSlots.set(hashLocationToOverflowLocation(insertedHashSlot), insertedHashSlotPosition); - } else { - statePositionInInsertedHashSlots.set(insertedHashSlot, insertedHashSlotPosition); - } - ++insertedHashSlotPosition; - } - } - insertedHashSlotsLength.setValue(insertedHashSlotPosition); - // endregion copy hash slots - hashSlotOffset += chunkOk.size(); - } - // region post build loop - sourceChunkIndexKeys.close(); - sourceChunkInsertedHashSlots.close(); - // endregion post build loop - } - } - - // mixin rehash - public void doRehash(BuildContext bc - // region extra rehash arguments - , WritableIntChunk sourceChunkInsertedHashSlots - , IntegerArraySource insertedHashSlots - // endregion extra rehash arguments - ) { - long firstBackingChunkLocation; - long lastBackingChunkLocation;// mixin rehash - // region rehash start - // endregion rehash start - while (rehashRequired()) { - // region rehash loop start - // endregion rehash loop start - if (tableHashPivot == tableSize) { - tableSize *= 2; - ensureCapacity(tableSize); - // region rehash ensure capacity - // endregion rehash ensure capacity - } - - final long targetBuckets = Math.min(MAX_TABLE_SIZE, (long)(numEntries / targetLoadFactor)); - final int bucketsToAdd = Math.max(1, (int)Math.min(Math.min(targetBuckets, tableSize) - tableHashPivot, bc.chunkSize)); - - initializeRehashLocations(bc.rehashLocations, bucketsToAdd); - - // fill the overflow bucket locations - overflowLocationSource.fillChunk(bc.overflowFillContext, bc.overflowLocations, RowSequenceFactory.wrapRowKeysChunkAsRowSequence(LongChunk.downcast(bc.rehashLocations))); - // null out the overflow locations in the table - setOverflowLocationsToNull(tableHashPivot - (tableSize >> 1), bucketsToAdd); - - while (bc.overflowLocations.size() > 0) { - // figure out which table location each overflow location maps to - compactOverflowLocations(bc.overflowLocations, bc.overflowLocationsToFetch); - if (bc.overflowLocationsToFetch.size() == 0) { - break; - } - - fillOverflowKeys(bc.overflowContexts, bc.workingKeyChunks, bc.overflowLocationsToFetch); - hashKeyChunks(bc.hashChunk, bc.workingKeyChunks); - convertHashToTableLocations(bc.hashChunk, bc.tableLocationsChunk, tableHashPivot + bucketsToAdd); - - // read the next chunk of overflow locations, which we will be overwriting in the next step - overflowOverflowLocationSource.fillChunkUnordered(bc.overflowOverflowFillContext, bc.overflowLocations, bc.overflowLocationsToFetch); - - // swap the table's overflow pointer with our location - swapOverflowPointers(bc.tableLocationsChunk, bc.overflowLocationsToFetch); - } - - // now rehash the main entries - - rowSetSource.fillChunkUnordered(bc.stateSourceFillContext, bc.workingStateEntries, bc.rehashLocations); - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(bc.workingStateEntries, EMPTY_VALUE, bc.shouldMoveBucket); - - // crush down things that don't exist - LongCompactKernel.compact(bc.rehashLocations, bc.shouldMoveBucket); - - // get the keys from the table - fillKeys(bc.workingFillContexts, bc.workingKeyChunks, bc.rehashLocations); - hashKeyChunks(bc.hashChunk, bc.workingKeyChunks); - convertHashToTableLocations(bc.hashChunk, bc.tableLocationsChunk, tableHashPivot + bucketsToAdd); - - // figure out which ones must move - LongChunkEquals.notEqual(bc.tableLocationsChunk, bc.rehashLocations, bc.shouldMoveBucket); - - firstBackingChunkLocation = -1; - lastBackingChunkLocation = -1; - // flushWriteThrough will have zero-ed out the sourcePositions and destinationLocationPositionInWriteThrough size - - int moves = 0; - for (int ii = 0; ii < bc.shouldMoveBucket.size(); ++ii) { - if (bc.shouldMoveBucket.get(ii)) { - moves++; - final long newHashLocation = bc.tableLocationsChunk.get(ii); - final long oldHashLocation = bc.rehashLocations.get(ii); - - if (newHashLocation > lastBackingChunkLocation) { - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, newHashLocation, keySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - } - - // @StateValueType@ from \QObject\E - final Object stateValueToMove = rowSetSource.getUnsafe(oldHashLocation); - rowSetSource.set(newHashLocation, stateValueToMove); - rowSetSource.set(oldHashLocation, EMPTY_VALUE); - // region rehash move values - final int positionInInsertedHashSlots = statePositionInInsertedHashSlots.getInt(oldHashLocation); - statePositionInInsertedHashSlots.set(newHashLocation, positionInInsertedHashSlots); - if (isChunkPosition(positionInInsertedHashSlots)) { - final int chunkPosition = insertedHashSlotPositionToChunkPosition(positionInInsertedHashSlots); - sourceChunkInsertedHashSlots.set(chunkPosition, (int) newHashLocation); - } else { - insertedHashSlots.set(positionInInsertedHashSlots, (int) newHashLocation); - } - // endregion rehash move values - - bc.sourcePositions.add(ii); - bc.destinationLocationPositionInWriteThrough.add((int)(newHashLocation - firstBackingChunkLocation)); - } - } - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - - // everything has been rehashed now, but we have some table locations that might have an overflow, - // without actually having a main entry. We walk through the empty main entries, pulling non-empty - // overflow locations into the main table - - // figure out which of the two possible locations is empty, because (1) we moved something from it - // or (2) we did not move something to it - bc.overflowLocationsToFetch.setSize(bc.shouldMoveBucket.size()); - final int totalPromotionsToProcess = bc.shouldMoveBucket.size(); - createOverflowPartitions(bc.overflowLocationsToFetch, bc.rehashLocations, bc.shouldMoveBucket, moves); - - for (int loop = 0; loop < 2; loop++) { - final boolean firstLoop = loop == 0; - - if (firstLoop) { - bc.overflowLocationForPromotionLoop.resetFromTypedChunk(bc.overflowLocationsToFetch, 0, moves); - } else { - bc.overflowLocationForPromotionLoop.resetFromTypedChunk(bc.overflowLocationsToFetch, moves, totalPromotionsToProcess - moves); - } - - overflowLocationSource.fillChunk(bc.overflowFillContext, bc.overflowLocations, RowSequenceFactory.wrapRowKeysChunkAsRowSequence(bc.overflowLocationForPromotionLoop)); - IntChunkEquals.notEqual(bc.overflowLocations, QueryConstants.NULL_INT, bc.shouldMoveBucket); - - // crunch the chunk down to relevant locations - LongCompactKernel.compact(bc.overflowLocationForPromotionLoop, bc.shouldMoveBucket); - IntCompactKernel.compact(bc.overflowLocations, bc.shouldMoveBucket); - - IntToLongCast.castInto(IntChunk.downcast(bc.overflowLocations), bc.overflowLocationsAsKeyIndices); - - // now fetch the overflow key values - fillOverflowKeys(bc.overflowContexts, bc.workingKeyChunks, bc.overflowLocationsAsKeyIndices); - // and their state values - overflowRowSetSource.fillChunkUnordered(bc.overflowStateSourceFillContext, bc.workingStateEntries, bc.overflowLocationsAsKeyIndices); - // and where their next pointer is - overflowOverflowLocationSource.fillChunkUnordered(bc.overflowOverflowFillContext, bc.overflowLocationsToMigrate, bc.overflowLocationsAsKeyIndices); - - // we'll have two sorted regions intermingled in the overflowLocationsToFetch, one of them is before the pivot, the other is after the pivot - // so that we can use our write through chunks, we first process the things before the pivot; then have a separate loop for those - // that go after - firstBackingChunkLocation = -1; - lastBackingChunkLocation = -1; - - for (int ii = 0; ii < bc.overflowLocationForPromotionLoop.size(); ++ii) { - final long tableLocation = bc.overflowLocationForPromotionLoop.get(ii); - if ((firstLoop && tableLocation < tableHashPivot) || (!firstLoop && tableLocation >= tableHashPivot)) { - if (tableLocation > lastBackingChunkLocation) { - if (bc.sourcePositions.size() > 0) { - // the permutes here are flushing the write through for the state and overflow locations - - IntPermuteKernel.permute(bc.sourcePositions, bc.overflowLocationsToMigrate, bc.destinationLocationPositionInWriteThrough, bc.writeThroughOverflowLocations); - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - } - - firstBackingChunkLocation = updateWriteThroughChunks(bc.writeThroughChunks, tableLocation, keySources); - lastBackingChunkLocation = firstBackingChunkLocation + bc.writeThroughChunks[0].size() - 1; - updateWriteThroughOverflow(bc.writeThroughOverflowLocations, firstBackingChunkLocation, lastBackingChunkLocation); - } - bc.sourcePositions.add(ii); - bc.destinationLocationPositionInWriteThrough.add((int)(tableLocation - firstBackingChunkLocation)); - // region promotion move - final long overflowLocation = bc.overflowLocationsAsKeyIndices.get(ii); - rowSetSource.set(tableLocation, overflowRowSetSource.get(overflowLocation)); - overflowRowSetSource.set(overflowLocation, EMPTY_VALUE); - final int positionInInsertedHashSlots = overflowStatePositionInInsertedHashSlots.getInt(overflowLocation); - statePositionInInsertedHashSlots.set(tableLocation, positionInInsertedHashSlots); - if (isChunkPosition(positionInInsertedHashSlots)) { - final int chunkPosition = insertedHashSlotPositionToChunkPosition(positionInInsertedHashSlots); - sourceChunkInsertedHashSlots.set(chunkPosition, (int) tableLocation); - } else { - insertedHashSlots.set(positionInInsertedHashSlots, (int) tableLocation); - } - // endregion promotion move - } - } - - // the permutes are completing the state and overflow promotions write through - IntPermuteKernel.permute(bc.sourcePositions, bc.overflowLocationsToMigrate, bc.destinationLocationPositionInWriteThrough, bc.writeThroughOverflowLocations); - flushWriteThrough(bc.sourcePositions, bc.workingKeyChunks, bc.destinationLocationPositionInWriteThrough, bc.writeThroughChunks); - - // now mark these overflow locations as free, so that we can reuse them - freeOverflowLocations.ensureCapacity(freeOverflowCount + bc.overflowLocations.size()); - // by sorting them, they will be more likely to be in the same write through chunk when we pull them from the free list - bc.overflowLocations.sort(); - for (int ii = 0; ii < bc.overflowLocations.size(); ++ii) { - freeOverflowLocations.set(freeOverflowCount++, bc.overflowLocations.get(ii)); - } - nullOverflowObjectSources(bc.overflowLocations); - } - - tableHashPivot += bucketsToAdd; - // region rehash loop end - // endregion rehash loop end - } - // region rehash final - // endregion rehash final - } - - public boolean rehashRequired() { - return numEntries > (tableHashPivot * maximumLoadFactor) && tableHashPivot < MAX_TABLE_SIZE; - } - - /** - * This function can be stuck in for debugging if you are breaking the table to make sure each slot still corresponds - * to the correct location. - */ - @SuppressWarnings({"unused", "unchecked"}) - private void verifyKeyHashes() { - final int maxSize = tableHashPivot; - - final ChunkSource.FillContext [] keyFillContext = makeFillContexts(keySources, SharedContext.makeSharedContext(), maxSize); - final WritableChunk [] keyChunks = getWritableKeyChunks(maxSize); - - try (final WritableLongChunk positions = WritableLongChunk.makeWritableChunk(maxSize); - final WritableBooleanChunk exists = WritableBooleanChunk.makeWritableChunk(maxSize); - final WritableIntChunk hashChunk = WritableIntChunk.makeWritableChunk(maxSize); - final WritableLongChunk tableLocationsChunk = WritableLongChunk.makeWritableChunk(maxSize); - final SafeCloseableArray ignored = new SafeCloseableArray<>(keyFillContext); - final SafeCloseableArray ignored2 = new SafeCloseableArray<>(keyChunks); - // @StateChunkName@ from \QObjectChunk\E - final WritableObjectChunk stateChunk = WritableObjectChunk.makeWritableChunk(maxSize); - final ChunkSource.FillContext fillContext = rowSetSource.makeFillContext(maxSize)) { - - rowSetSource.fillChunk(fillContext, stateChunk, RowSetFactory.flat(tableHashPivot)); - - ChunkUtils.fillInOrder(positions); - - // @StateChunkIdentityName@ from \QObjectChunkIdentity\E - ObjectChunkIdentityEquals.notEqual(stateChunk, EMPTY_VALUE, exists); - - // crush down things that don't exist - LongCompactKernel.compact(positions, exists); - - // get the keys from the table - fillKeys(keyFillContext, keyChunks, positions); - hashKeyChunks(hashChunk, keyChunks); - convertHashToTableLocations(hashChunk, tableLocationsChunk, tableHashPivot); - - for (int ii = 0; ii < positions.size(); ++ii) { - if (tableLocationsChunk.get(ii) != positions.get(ii)) { - throw new IllegalStateException(); - } - } - } - } - - void setTargetLoadFactor(final double targetLoadFactor) { - this.targetLoadFactor = targetLoadFactor; - } - - void setMaximumLoadFactor(final double maximumLoadFactor) { - this.maximumLoadFactor = maximumLoadFactor; - } - - private void createOverflowPartitions(WritableLongChunk overflowLocationsToFetch, WritableLongChunk rehashLocations, WritableBooleanChunk shouldMoveBucket, int moves) { - int startWritePosition = 0; - int endWritePosition = moves; - for (int ii = 0; ii < shouldMoveBucket.size(); ++ii) { - if (shouldMoveBucket.get(ii)) { - final long oldHashLocation = rehashLocations.get(ii); - // this needs to be promoted, because we moved it - overflowLocationsToFetch.set(startWritePosition++, oldHashLocation); - } else { - // we didn't move anything into the destination slot; so we need to promote its overflow - final long newEmptyHashLocation = rehashLocations.get(ii) + (tableSize >> 1); - overflowLocationsToFetch.set(endWritePosition++, newEmptyHashLocation); - } - } - } - - private void setOverflowLocationsToNull(long start, int count) { - for (int ii = 0; ii < count; ++ii) { - overflowLocationSource.set(start + ii, QueryConstants.NULL_INT); - } - } - - private void initializeRehashLocations(WritableLongChunk rehashLocations, int bucketsToAdd) { - rehashLocations.setSize(bucketsToAdd); - for (int ii = 0; ii < bucketsToAdd; ++ii) { - rehashLocations.set(ii, tableHashPivot + ii - (tableSize >> 1)); - } - } - - private void compactOverflowLocations(IntChunk overflowLocations, WritableLongChunk overflowLocationsToFetch) { - overflowLocationsToFetch.setSize(0); - for (int ii = 0; ii < overflowLocations.size(); ++ii) { - final int overflowLocation = overflowLocations.get(ii); - if (overflowLocation != QueryConstants.NULL_INT) { - overflowLocationsToFetch.add(overflowLocation); - } - } - } - - private void swapOverflowPointers(LongChunk tableLocationsChunk, LongChunk overflowLocationsToFetch) { - for (int ii = 0; ii < overflowLocationsToFetch.size(); ++ii) { - final long newLocation = tableLocationsChunk.get(ii); - final int existingOverflow = overflowLocationSource.getUnsafe(newLocation); - final long overflowLocation = overflowLocationsToFetch.get(ii); - overflowOverflowLocationSource.set(overflowLocation, existingOverflow); - overflowLocationSource.set(newLocation, (int)overflowLocation); - } - } - - - private void updateWriteThroughOverflow(ResettableWritableIntChunk writeThroughOverflow, long firstPosition, long expectedLastPosition) { - final long firstBackingChunkPosition = overflowLocationSource.resetWritableChunkToBackingStore(writeThroughOverflow, firstPosition); - if (firstBackingChunkPosition != firstPosition) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - if (firstBackingChunkPosition + writeThroughOverflow.size() - 1 != expectedLastPosition) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - } - - // endmixin rehash - - private int allocateOverflowLocation() { - // mixin rehash - if (freeOverflowCount > 0) { - return freeOverflowLocations.getUnsafe(--freeOverflowCount); - } - // endmixin rehash - return nextOverflowLocation++; - } - - private static long updateWriteThroughChunks(ResettableWritableChunk[] writeThroughChunks, long currentHashLocation, ArrayBackedColumnSource[] sources) { - final long firstBackingChunkPosition = sources[0].resetWritableChunkToBackingStore(writeThroughChunks[0], currentHashLocation); - for (int jj = 1; jj < sources.length; ++jj) { - if (sources[jj].resetWritableChunkToBackingStore(writeThroughChunks[jj], currentHashLocation) != firstBackingChunkPosition) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - if (writeThroughChunks[jj].size() != writeThroughChunks[0].size()) { - throw new IllegalStateException("ArrayBackedColumnSources have different block sizes!"); - } - } - return firstBackingChunkPosition; - } - - private void flushWriteThrough(WritableIntChunk sourcePositions, Chunk[] sourceKeyChunks, WritableIntChunk destinationLocationPositionInWriteThrough, WritableChunk[] writeThroughChunks) { - if (sourcePositions.size() < 0) { - return; - } - for (int jj = 0; jj < keySources.length; ++jj) { - chunkCopiers[jj].permute(sourcePositions, sourceKeyChunks[jj], destinationLocationPositionInWriteThrough, writeThroughChunks[jj]); - } - sourcePositions.setSize(0); - destinationLocationPositionInWriteThrough.setSize(0); - } - - // mixin rehash - private void nullOverflowObjectSources(IntChunk locationsToNull) { - for (ObjectArraySource objectArraySource : overflowKeyColumnsToNull) { - for (int ii = 0; ii < locationsToNull.size(); ++ii) { - objectArraySource.set(locationsToNull.get(ii), null); - } - } - // region nullOverflowObjectSources - // NB: It's worth nulling out the overflowRowSetSource because if it's never re-used its sequential builder will never be swapped to a RowSet - for (int ii = 0; ii < locationsToNull.size(); ++ii) { - overflowRowSetSource.set(locationsToNull.get(ii), EMPTY_VALUE); - } - // endregion nullOverflowObjectSources - } - // endmixin rehash - - private void checkKeyEquality(WritableBooleanChunk equalValues, WritableChunk[] workingKeyChunks, Chunk[] sourceKeyChunks) { - for (int ii = 0; ii < sourceKeyChunks.length; ++ii) { - chunkEquals[ii].andEqual(workingKeyChunks[ii], sourceKeyChunks[ii], equalValues); - } - } - - private void checkLhsPermutedEquality(WritableIntChunk chunkPositionsForFetches, Chunk[] sourceKeyChunks, WritableChunk[] overflowKeyChunks, WritableBooleanChunk equalValues) { - chunkEquals[0].equalLhsPermuted(chunkPositionsForFetches, sourceKeyChunks[0], overflowKeyChunks[0], equalValues); - for (int ii = 1; ii < overflowKeySources.length; ++ii) { - chunkEquals[ii].andEqualLhsPermuted(chunkPositionsForFetches, sourceKeyChunks[ii], overflowKeyChunks[ii], equalValues); - } - } - - private void checkPairEquality(WritableIntChunk chunkPositionsToCheckForEquality, Chunk[] sourceKeyChunks, WritableBooleanChunk equalPairs) { - chunkEquals[0].equalPairs(chunkPositionsToCheckForEquality, sourceKeyChunks[0], equalPairs); - for (int ii = 1; ii < keyColumnCount; ++ii) { - chunkEquals[ii].andEqualPairs(chunkPositionsToCheckForEquality, sourceKeyChunks[ii], equalPairs); - } - } - - private void fillKeys(ColumnSource.FillContext[] fillContexts, WritableChunk[] keyChunks, WritableLongChunk tableLocationsChunk) { - fillKeys(keySources, fillContexts, keyChunks, tableLocationsChunk); - } - - private void fillOverflowKeys(ColumnSource.FillContext[] fillContexts, WritableChunk[] keyChunks, WritableLongChunk overflowLocationsChunk) { - fillKeys(overflowKeySources, fillContexts, keyChunks, overflowLocationsChunk); - } - - private static void fillKeys(ArrayBackedColumnSource[] keySources, ColumnSource.FillContext[] fillContexts, WritableChunk[] keyChunks, WritableLongChunk keyIndices) { - for (int ii = 0; ii < keySources.length; ++ii) { - keySources[ii].fillChunkUnordered(fillContexts[ii], keyChunks[ii], keyIndices); - } - } - - private void hashKeyChunks(WritableIntChunk hashChunk, Chunk[] sourceKeyChunks) { - chunkHashers[0].hashInitial(sourceKeyChunks[0], hashChunk); - for (int ii = 1; ii < sourceKeyChunks.length; ++ii) { - chunkHashers[ii].hashUpdate(sourceKeyChunks[ii], hashChunk); - } - } - - private void getKeyChunks(ColumnSource[] sources, ColumnSource.GetContext[] contexts, Chunk[] chunks, RowSequence rowSequence) { - for (int ii = 0; ii < chunks.length; ++ii) { - chunks[ii] = sources[ii].getChunk(contexts[ii], rowSequence); - } - } - - - // region probe wrappers - // endregion probe wrappers - - - private void convertHashToTableLocations(WritableIntChunk hashChunk, WritableLongChunk tablePositionsChunk) { - // mixin rehash - // NOTE that this mixin section is a bit ugly, we are spanning the two functions so that we can avoid using tableHashPivot and having the unused pivotPoint parameter - convertHashToTableLocations(hashChunk, tablePositionsChunk, tableHashPivot); - } - - private void convertHashToTableLocations(WritableIntChunk hashChunk, WritableLongChunk tablePositionsChunk, int pivotPoint) { - // endmixin rehash - - // turn hash codes into indices within our table - for (int ii = 0; ii < hashChunk.size(); ++ii) { - final int hash = hashChunk.get(ii); - // mixin rehash - final int location = hashToTableLocation(pivotPoint, hash); - // endmixin rehash - // altmixin rehash: final int location = hashToTableLocation(hash); - tablePositionsChunk.set(ii, location); - } - tablePositionsChunk.setSize(hashChunk.size()); - } - - private int hashToTableLocation( - // mixin rehash - int pivotPoint, - // endmixin rehash - int hash) { - // altmixin rehash: final \ - int location = hash & (tableSize - 1); - // mixin rehash - if (location >= pivotPoint) { - location -= (tableSize >> 1); - } - // endmixin rehash - return location; - } - - // region extraction functions - void convertBuildersToIndexes(@NotNull final IntegerArraySource insertedHashSlots, final int insertedHashSlotsLength) { - for (int insertedHashSlotPosition = 0; insertedHashSlotPosition < insertedHashSlotsLength; ++insertedHashSlotPosition) { - final int insertedHashSlot = insertedHashSlots.getInt(insertedHashSlotPosition); - if (isOverflowLocation(insertedHashSlot)) { - final int insertedOverflowSlot = hashLocationToOverflowLocation(insertedHashSlot); - overflowRowSetSource.set(insertedOverflowSlot, ((RowSetBuilderSequential) overflowRowSetSource.get(insertedOverflowSlot)).build()); - } else { - rowSetSource.set(insertedHashSlot, ((RowSetBuilderSequential) rowSetSource.get(insertedHashSlot)).build()); - } - } - } - - ColumnSource[] getKeyHashTableSources() { - final ColumnSource[] keyHashTableSources = new ColumnSource[keyColumnCount]; - for (int kci = 0; kci < keyColumnCount; ++kci) { - // noinspection unchecked - keyHashTableSources[kci] = new HashTableColumnSource(keySources[kci], overflowKeySources[kci]); - } - return keyHashTableSources; - } - - ColumnSource getIndexHashTableSource() { - //noinspection unchecked - return new HashTableColumnSource(TrackingWritableRowSet.class, rowSetSource, overflowRowSetSource); - } - // endregion extraction functions - - @NotNull - private static ColumnSource.FillContext[] makeFillContexts(ColumnSource[] keySources, final SharedContext sharedContext, int chunkSize) { - final ColumnSource.FillContext[] workingFillContexts = new ColumnSource.FillContext[keySources.length]; - for (int ii = 0; ii < keySources.length; ++ii) { - workingFillContexts[ii] = keySources[ii].makeFillContext(chunkSize, sharedContext); - } - return workingFillContexts; - } - - private static ColumnSource.GetContext[] makeGetContexts(ColumnSource [] sources, final SharedContext sharedState, int chunkSize) { - final ColumnSource.GetContext[] contexts = new ColumnSource.GetContext[sources.length]; - for (int ii = 0; ii < sources.length; ++ii) { - contexts[ii] = sources[ii].makeGetContext(chunkSize, sharedState); - } - return contexts; - } - - @NotNull - private WritableChunk[] getWritableKeyChunks(int chunkSize) { - //noinspection unchecked - final WritableChunk[] workingKeyChunks = new WritableChunk[keyChunkTypes.length]; - for (int ii = 0; ii < keyChunkTypes.length; ++ii) { - workingKeyChunks[ii] = keyChunkTypes[ii].makeWritableChunk(chunkSize); - } - return workingKeyChunks; - } - - @NotNull - private ResettableWritableChunk[] getResettableWritableKeyChunks() { - //noinspection unchecked - final ResettableWritableChunk[] workingKeyChunks = new ResettableWritableChunk[keyChunkTypes.length]; - for (int ii = 0; ii < keyChunkTypes.length; ++ii) { - workingKeyChunks[ii] = keyChunkTypes[ii].makeResettableWritableChunk(); - } - return workingKeyChunks; - } - - // region getStateValue - // endregion getStateValue - - // region overflowLocationToHashLocation - static boolean isOverflowLocation(final int hashSlot) { - return HashTableColumnSource.isOverflowLocation(hashSlot); - } - - static int hashLocationToOverflowLocation(final int hashSlot) { - return HashTableColumnSource.hashLocationToOverflowLocation(hashSlot); - } - - private static int overflowLocationToHashLocation(final int overflowSlot) { - return (int)HashTableColumnSource.overflowLocationToHashLocation(overflowSlot); - } - - private static boolean isChunkPosition(final int insertedHashSlotPosition) { - return insertedHashSlotPosition < 0; - } - - private static int insertedHashSlotPositionToChunkPosition(final int insertedHashSlotPosition) { - return -(insertedHashSlotPosition + 1); - } - - private static int chunkPositionToInsertedHashSlotPosition(final int chunkPosition) { - return -chunkPosition - 1; - } - // endregion overflowLocationToHashLocation - - - static int hashTableSize(long initialCapacity) { - return (int)Math.max(MINIMUM_INITIAL_HASH_SIZE, Math.min(MAX_TABLE_SIZE, Long.highestOneBit(initialCapacity) * 2)); - } - -} diff --git a/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateHashTable.java b/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateHashTable.java index a8a64ccc5cd..73af998228f 100644 --- a/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateHashTable.java +++ b/replication/reflective/src/main/java/io/deephaven/replicators/ReplicateHashTable.java @@ -125,17 +125,6 @@ public static void main(String[] args) throws IOException, ClassNotFoundExceptio "engine/table/src/main/java/io/deephaven/engine/table/impl/IncrementalChunkedNaturalJoinStateManager.java", "engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedOperatorAggregationStateManager.java", allowMissingDestinations, Collections.singletonList("dumpTable")); - - // Incremental NJ -> Incremental By (non-operator) - doReplicate( - "engine/table/src/main/java/io/deephaven/engine/table/impl/IncrementalChunkedNaturalJoinStateManager.java", - "engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.java", - allowMissingDestinations, Arrays.asList("dumpTable", "allowUpdateWriteThroughState")); - // Incremental By -> Static By (non-operator) - doReplicate( - "engine/table/src/main/java/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.java", - "engine/table/src/main/java/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.java", - allowMissingDestinations, Arrays.asList("dumpTable", "prev", "decorationProbe")); } private static class RegionedFile { From 38a8d24db9ec2a5a8957592e1ffcae864b1f6c98 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 11:11:50 -0500 Subject: [PATCH 29/44] Freeze-by support --- .../table/impl/AggAllByCopyAttributes.java | 27 +------ .../engine/table/impl/AggAllByUseTable.java | 29 ++----- .../table/impl/by/AggregationProcessor.java | 27 ++++++- .../impl/by/FreezeByAggregationFactory.java | 77 ------------------- .../engine/table/impl/util/FreezeBy.java | 13 ++-- .../client/impl/BatchTableRequestBuilder.java | 6 ++ .../qst/table/AggAllByExclusions.java | 29 ++----- .../io/deephaven/api/agg/spec/AggSpec.java | 6 ++ .../api/agg/spec/AggSpecColumnReferences.java | 5 ++ .../deephaven/api/agg/spec/AggSpecFreeze.java | 32 ++++++++ 10 files changed, 96 insertions(+), 155 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java index 21cdf079e1b..6144010ceea 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java @@ -1,28 +1,6 @@ package io.deephaven.engine.table.impl; -import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.api.agg.spec.AggSpecAbsSum; -import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; -import io.deephaven.api.agg.spec.AggSpecAvg; -import io.deephaven.api.agg.spec.AggSpecCountDistinct; -import io.deephaven.api.agg.spec.AggSpecDistinct; -import io.deephaven.api.agg.spec.AggSpecFirst; -import io.deephaven.api.agg.spec.AggSpecFormula; -import io.deephaven.api.agg.spec.AggSpecGroup; -import io.deephaven.api.agg.spec.AggSpecLast; -import io.deephaven.api.agg.spec.AggSpecMax; -import io.deephaven.api.agg.spec.AggSpecMedian; -import io.deephaven.api.agg.spec.AggSpecMin; -import io.deephaven.api.agg.spec.AggSpecPercentile; -import io.deephaven.api.agg.spec.AggSpecSortedFirst; -import io.deephaven.api.agg.spec.AggSpecSortedLast; -import io.deephaven.api.agg.spec.AggSpecStd; -import io.deephaven.api.agg.spec.AggSpecSum; -import io.deephaven.api.agg.spec.AggSpecTDigest; -import io.deephaven.api.agg.spec.AggSpecUnique; -import io.deephaven.api.agg.spec.AggSpecVar; -import io.deephaven.api.agg.spec.AggSpecWAvg; -import io.deephaven.api.agg.spec.AggSpecWSum; +import io.deephaven.api.agg.spec.*; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.BaseTable.CopyAttributeOperation; @@ -50,6 +28,9 @@ public void visit(AggSpecCountDistinct countDistinct) {} @Override public void visit(AggSpecDistinct distinct) {} + @Override + public void visit(AggSpecFreeze freeze) {} + @Override public void visit(AggSpecGroup group) {} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java index 5be88f67e43..70bc8b18727 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java @@ -1,28 +1,6 @@ package io.deephaven.engine.table.impl; -import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.api.agg.spec.AggSpecAbsSum; -import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; -import io.deephaven.api.agg.spec.AggSpecAvg; -import io.deephaven.api.agg.spec.AggSpecCountDistinct; -import io.deephaven.api.agg.spec.AggSpecDistinct; -import io.deephaven.api.agg.spec.AggSpecFirst; -import io.deephaven.api.agg.spec.AggSpecFormula; -import io.deephaven.api.agg.spec.AggSpecGroup; -import io.deephaven.api.agg.spec.AggSpecLast; -import io.deephaven.api.agg.spec.AggSpecMax; -import io.deephaven.api.agg.spec.AggSpecMedian; -import io.deephaven.api.agg.spec.AggSpecMin; -import io.deephaven.api.agg.spec.AggSpecPercentile; -import io.deephaven.api.agg.spec.AggSpecSortedFirst; -import io.deephaven.api.agg.spec.AggSpecSortedLast; -import io.deephaven.api.agg.spec.AggSpecStd; -import io.deephaven.api.agg.spec.AggSpecSum; -import io.deephaven.api.agg.spec.AggSpecTDigest; -import io.deephaven.api.agg.spec.AggSpecUnique; -import io.deephaven.api.agg.spec.AggSpecVar; -import io.deephaven.api.agg.spec.AggSpecWAvg; -import io.deephaven.api.agg.spec.AggSpecWSum; +import io.deephaven.api.agg.spec.*; import io.deephaven.engine.table.Table; import java.util.Objects; @@ -72,6 +50,11 @@ public void visit(AggSpecDistinct distinct) { keep(); } + @Override + public void visit(AggSpecFreeze freeze) { + keep(); + } + @Override public void visit(AggSpecGroup group) { drop(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 2cac102bce5..b42d28659e4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -18,6 +18,7 @@ import io.deephaven.api.agg.spec.AggSpecDistinct; import io.deephaven.api.agg.spec.AggSpecFirst; import io.deephaven.api.agg.spec.AggSpecFormula; +import io.deephaven.api.agg.spec.AggSpecFreeze; import io.deephaven.api.agg.spec.AggSpecGroup; import io.deephaven.api.agg.spec.AggSpecLast; import io.deephaven.api.agg.spec.AggSpecMax; @@ -102,6 +103,8 @@ import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.sources.SingleValueObjectColumnSource; import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; +import io.deephaven.engine.table.impl.util.freezeby.FreezeByCountOperator; +import io.deephaven.engine.table.impl.util.freezeby.FreezeByOperator; import io.deephaven.time.DateTime; import io.deephaven.util.FunctionalInterfaces.TriFunction; import io.deephaven.util.annotations.FinalDefault; @@ -281,6 +284,7 @@ private abstract class Converter implements Aggregation.Visitor, AggSpec.Visitor final List transformers = new ArrayList<>(); List resultPairs = List.of(); + int freezeByCountIndex = -1; int trackedFirstOrLastIndex = -1; private Converter(@NotNull final Table table, @NotNull final String... groupByColumnNames) { @@ -393,7 +397,7 @@ final void addApproximatePercentileOperator(final double percentile, final doubl (operator = operators.get(ii)) instanceof TDigestPercentileOperator) { final TDigestPercentileOperator tDigestOperator = (TDigestPercentileOperator) operator; if (tDigestOperator.compression() == compression) { - addOperator(tDigestOperator.makeSecondaryOperator(percentile, resultName), inputSource, + addOperator(tDigestOperator.makeSecondaryOperator(percentile, resultName), null, inputName); return; } @@ -403,6 +407,17 @@ final void addApproximatePercentileOperator(final double percentile, final doubl inputName); } + final void addFreezeOperators() { + final FreezeByCountOperator countOperator; + if (freezeByCountIndex >= 0) { + countOperator = (FreezeByCountOperator) operators.get(freezeByCountIndex); + } else { + freezeByCountIndex = operators.size(); + addNoInputOperator(countOperator = new FreezeByCountOperator()); + } + addBasicOperators((t, n) -> new FreezeByOperator(t, n, countOperator)); + } + final void addMinOrMaxOperators(final boolean isMin) { for (final Pair pair : resultPairs) { final String inputName = pair.input().name(); @@ -624,6 +639,11 @@ public void visit(@NotNull final AggSpecDistinct distinct) { addBasicOperators((t, n) -> makeDistinctOperator(t, n, distinct.includeNulls(), false, false)); } + @Override + public void visit(AggSpecFreeze freeze) { + addFreezeOperators(); + } + @Override public void visit(@NotNull final AggSpecGroup group) { streamUnsupported("Group"); @@ -754,6 +774,11 @@ default void visit(@NotNull final AggSpecApproximatePercentile approxPct) { rollupUnsupported("ApproximatePercentile"); } + @Override + default void visit(AggSpecFreeze freeze) { + rollupUnsupported("Freeze"); + } + @Override @FinalDefault default void visit(@NotNull final AggSpecGroup group) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.java deleted file mode 100644 index 40ad3fc6044..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Copyright (c) 2020 Deephaven Data Labs and Patent Pending - */ - -package io.deephaven.engine.table.impl.by; - -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.Table; -import io.deephaven.time.DateTime; -import io.deephaven.engine.table.ColumnSource; -import io.deephaven.engine.table.impl.sources.ReinterpretUtils; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import io.deephaven.engine.table.impl.util.freezeby.FreezeByCountOperator; -import io.deephaven.engine.table.impl.util.freezeby.FreezeByOperator; -import org.jetbrains.annotations.NotNull; - -import java.util.*; - -public class FreezeByAggregationFactory implements AggregationContextFactory { - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - return getAllColumnOperators(table, groupByColumns); - } - - private static AggregationContext getAllColumnOperators(Table withView, String[] groupByNameArray) { - final Set groupByNames = new HashSet<>(Arrays.asList(groupByNameArray)); - final int operatorCount = withView.getColumnSourceMap().size() - groupByNames.size() + 1; - - final List operators = new ArrayList<>(operatorCount); - final List> inputColumns = new ArrayList<>(operatorCount); - final List inputNames = new ArrayList<>(operatorCount - 1); - - final FreezeByCountOperator countOperator = new FreezeByCountOperator(); - inputColumns.add(null); - operators.add(countOperator); - - withView.getColumnSourceMap().forEach((name, columnSource) -> { - if (groupByNames.contains(name)) { - return; - } - - final Class type = columnSource.getType(); - - // For DateTime columns, the in-memory source uses longs internally, and all supported aggregations (i.e. - // min and max) work correctly against longs. - final ColumnSource inputSource = - columnSource.getType() == DateTime.class ? ReinterpretUtils.dateTimeToLongSource(columnSource) - : columnSource; - - // noinspection unchecked - inputColumns.add(inputSource); - inputNames.add(name); - operators.add(new FreezeByOperator(type, name, countOperator)); - }); - - final String[][] inputNameArray = new String[inputNames.size() + 1][1]; - inputNameArray[0] = CollectionUtil.ZERO_LENGTH_STRING_ARRAY; - for (int ii = 0; ii < inputNames.size(); ++ii) { - inputNameArray[ii + 1][0] = inputNames.get(ii); - } - - // noinspection unchecked - return new AggregationContext( - operators.toArray( - IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY), - inputNameArray, - inputColumns.toArray(ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY)); - } - - @Override - public String toString() { - return "FreezeBy"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/FreezeBy.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/FreezeBy.java index 228d903c1ea..96aa3f18720 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/util/FreezeBy.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/util/FreezeBy.java @@ -1,9 +1,7 @@ package io.deephaven.engine.table.impl.util; +import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.select.SelectColumnFactory; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.by.*; import io.deephaven.engine.table.impl.select.SelectColumn; import java.util.Collection; @@ -34,7 +32,7 @@ private FreezeBy() {} // static use only * @return a frozen copy of the input table */ public static Table freezeBy(Table input) { - return freezeBy(input, SelectColumn.ZERO_LENGTH_SELECT_COLUMN_ARRAY); + return input.aggAllBy(AggSpec.freeze()); } /** @@ -51,7 +49,7 @@ public static Table freezeBy(Table input) { * @return a copy of the input table frozen by key */ public static Table freezeBy(Table input, String... groupByColumns) { - return freezeBy(input, SelectColumnFactory.getExpressions(groupByColumns)); + return input.aggAllBy(AggSpec.freeze(), groupByColumns); } /** @@ -68,7 +66,7 @@ public static Table freezeBy(Table input, String... groupByColumns) { * @return a copy of the input table frozen by key */ public static Table freezeBy(Table input, Collection groupByColumns) { - return freezeBy(input, SelectColumnFactory.getExpressions(groupByColumns)); + return input.aggAllBy(AggSpec.freeze(), groupByColumns); } /** @@ -85,7 +83,6 @@ public static Table freezeBy(Table input, Collection groupByColumns) { * @return a copy of the input table frozen by key */ public static Table freezeBy(Table input, SelectColumn... groupByColumns) { - return ChunkedOperatorAggregationHelper.aggregation(new FreezeByAggregationFactory(), - (QueryTable) input.coalesce(), groupByColumns); + return input.aggAllBy(AggSpec.freeze(), groupByColumns); } } diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index 16a29dbd708..2c3ba896fdd 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -697,6 +697,12 @@ public void visit(AggSpecDistinct distinct) { "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } + @Override + public void visit(AggSpecFreeze freeze) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + } + @Override public void visit(AggSpecGroup group) { out = of(AggType.GROUP, pairs).build(); diff --git a/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java b/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java index 57e8660ba19..e9996fea9a2 100644 --- a/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java +++ b/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java @@ -2,30 +2,8 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.Selectable; -import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.api.agg.spec.*; import io.deephaven.api.agg.spec.AggSpec.Visitor; -import io.deephaven.api.agg.spec.AggSpecAbsSum; -import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; -import io.deephaven.api.agg.spec.AggSpecAvg; -import io.deephaven.api.agg.spec.AggSpecCountDistinct; -import io.deephaven.api.agg.spec.AggSpecDistinct; -import io.deephaven.api.agg.spec.AggSpecFirst; -import io.deephaven.api.agg.spec.AggSpecFormula; -import io.deephaven.api.agg.spec.AggSpecGroup; -import io.deephaven.api.agg.spec.AggSpecLast; -import io.deephaven.api.agg.spec.AggSpecMax; -import io.deephaven.api.agg.spec.AggSpecMedian; -import io.deephaven.api.agg.spec.AggSpecMin; -import io.deephaven.api.agg.spec.AggSpecPercentile; -import io.deephaven.api.agg.spec.AggSpecSortedFirst; -import io.deephaven.api.agg.spec.AggSpecSortedLast; -import io.deephaven.api.agg.spec.AggSpecStd; -import io.deephaven.api.agg.spec.AggSpecSum; -import io.deephaven.api.agg.spec.AggSpecTDigest; -import io.deephaven.api.agg.spec.AggSpecUnique; -import io.deephaven.api.agg.spec.AggSpecVar; -import io.deephaven.api.agg.spec.AggSpecWAvg; -import io.deephaven.api.agg.spec.AggSpecWSum; import java.util.Collection; import java.util.Collections; @@ -72,6 +50,11 @@ public void visit(AggSpecDistinct distinct) { out = Collections.emptySet(); } + @Override + public void visit(AggSpecFreeze freeze) { + out = Collections.emptySet(); + } + @Override public void visit(AggSpecGroup group) { out = Collections.emptySet(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 150cb01d57a..2dd1b65dbc9 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -58,6 +58,10 @@ static AggSpecFormula formula(String formula, String formulaParam) { return AggSpecFormula.of(formula, formulaParam); } + static AggSpecFreeze freeze() { + return AggSpecFreeze.of(); + } + static AggSpecGroup group() { return AggSpecGroup.of(); } @@ -169,6 +173,8 @@ interface Visitor { void visit(AggSpecDistinct distinct); + void visit(AggSpecFreeze freeze); + void visit(AggSpecGroup group); void visit(AggSpecAvg avg); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java index 068fca2f0f2..aa8b96f31f5 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java @@ -40,6 +40,11 @@ public void visit(AggSpecDistinct distinct) { out = Collections.emptySet(); } + @Override + public void visit(AggSpecFreeze freeze) { + out = Collections.emptySet(); + } + @Override public void visit(AggSpecGroup group) { out = Collections.emptySet(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java new file mode 100644 index 00000000000..a3d21d828be --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java @@ -0,0 +1,32 @@ +package io.deephaven.api.agg.spec; + +import io.deephaven.annotations.SimpleStyle; +import org.immutables.value.Value.Immutable; + +/** + * Aggregation that freezes the first value for each group and ignores subsequent changes. When groups are removed, the + * corresponding output row is removed. When groups are re-added (on a subsequent update cycle), the newly added value + * is then frozen. + *

+ * Only one row per group is allowed in the output, because the operation has no way to determine which row to freeze + * otherwise. This is a constraint on the input data. + */ +@Immutable +@SimpleStyle +public abstract class AggSpecFreeze extends AggSpecEmptyBase { + + public static AggSpecFreeze of() { + return ImmutableAggSpecFreeze.of(); + } + + @Override + public final String description() { + return "freeze"; + } + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } +} From eeaf4accc14ba63bd7fdde6251a95cb001387978 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 11:53:54 -0500 Subject: [PATCH 30/44] Delete more aggregation factories. --- .../by/FirstOrLastByAggregationFactory.java | 64 ------- .../impl/by/FormulaAggregationFactory.java | 88 --------- .../impl/by/GroupByAggregationFactory.java | 70 ------- .../impl/by/KeyOnlyAggregationFactory.java | 21 --- .../WeightedAverageSumAggregationFactory.java | 176 ------------------ .../table/impl/QueryTableAggregationTest.java | 32 +++- 6 files changed, 24 insertions(+), 427 deletions(-) delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java delete mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.java deleted file mode 100644 index bf5891f9e72..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.java +++ /dev/null @@ -1,64 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.api.util.NameValidator; -import io.deephaven.engine.table.impl.BaseTable; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.ChunkSource; -import org.jetbrains.annotations.NotNull; - -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; - -public class FirstOrLastByAggregationFactory implements AggregationContextFactory { - private final boolean isFirst; - private final String exposeRedirection; - - public FirstOrLastByAggregationFactory(boolean isFirst) { - this(isFirst, null); - } - - public FirstOrLastByAggregationFactory(boolean isFirst, String exposeRedirection) { - this.isFirst = isFirst; - this.exposeRedirection = exposeRedirection == null ? null : NameValidator.validateColumnName(exposeRedirection); - } - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - // noinspection unchecked - final ChunkSource.WithPrev[] inputSource = new ChunkSource.WithPrev[1]; - inputSource[0] = null; - - final IterativeChunkedAggregationOperator[] operator = new IterativeChunkedAggregationOperator[1]; - final String[][] name = new String[1][0]; - name[0] = CollectionUtil.ZERO_LENGTH_STRING_ARRAY; - - final Set groupBySet = new HashSet<>(Arrays.asList(groupByColumns)); - final MatchPair[] matchPairs = table.getDefinition().getColumnNames().stream() - .filter(col -> !groupBySet.contains(col)).map(col -> new MatchPair(col, col)).toArray(MatchPair[]::new); - - if (table.isRefreshing()) { - if (((BaseTable) table).isStream()) { - operator[0] = isFirst ? new StreamFirstChunkedOperator(matchPairs, table) - : new StreamLastChunkedOperator(matchPairs, table); - } else if (((BaseTable) table).isAddOnly()) { - operator[0] = new AddOnlyFirstOrLastChunkedOperator(isFirst, matchPairs, table, exposeRedirection); - } else { - operator[0] = new FirstOrLastChunkedOperator(isFirst, matchPairs, table, exposeRedirection); - } - } else { - operator[0] = new StaticFirstOrLastChunkedOperator(isFirst, matchPairs, table, exposeRedirection); - } - - return new AggregationContext(operator, name, inputSource); - } - - @Override - public String toString() { - return isFirst ? "FirstBy" : "LastBy"; - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.java deleted file mode 100644 index 66a094fa1a2..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.java +++ /dev/null @@ -1,88 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.MatchPair; -import io.deephaven.engine.table.impl.select.MatchPairFactory; -import io.deephaven.engine.table.impl.select.SelectColumnFactory; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.select.SelectColumn; -import io.deephaven.engine.table.ChunkSource; -import org.jetbrains.annotations.NotNull; - -import java.util.Arrays; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * An {@link AggregationContextFactory} used in the implementation of {@link Table#applyToAllBy}. - */ -public class FormulaAggregationFactory implements AggregationContextFactory { - - private final String formula; - private final String columnParamName; - - public FormulaAggregationFactory(@NotNull final String formula, @NotNull final String columnParamName) { - this.formula = formula; - this.columnParamName = columnParamName; - } - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table inputTable, - @NotNull final String... groupByColumnNames) { - final Set groupByColumnNameSet = Arrays.stream(groupByColumnNames).collect(Collectors.toSet()); - final MatchPair[] resultColumns = - inputTable.getDefinition().getColumnNames().stream().filter(cn -> !groupByColumnNameSet.contains(cn)) - .map(MatchPairFactory::getExpression).toArray(MatchPair[]::new); - - final GroupByChunkedOperator groupByChunkedOperator = - new GroupByChunkedOperator((QueryTable) inputTable, false, resultColumns); - final FormulaChunkedOperator formulaChunkedOperator = - new FormulaChunkedOperator(groupByChunkedOperator, true, formula, columnParamName, resultColumns); - - // noinspection unchecked - return new AggregationContext( - new IterativeChunkedAggregationOperator[] {formulaChunkedOperator}, - new String[][] {CollectionUtil.ZERO_LENGTH_STRING_ARRAY}, - new ChunkSource.WithPrev[] {null}); - } - - @Override - public String toString() { - return "ApplyToAllBy(" + formula + ", " + columnParamName + ')'; - } - - public static QueryTable applyToAllBy(@NotNull final QueryTable inputTable, - @NotNull final String formula, - @NotNull final String columnParamName, - @NotNull final String... groupByColumnNames) { - return applyToAllBy(AggregationControl.DEFAULT_FOR_OPERATOR, inputTable, formula, columnParamName, - groupByColumnNames); - } - - public static QueryTable applyToAllBy(@NotNull final QueryTable inputTable, - @NotNull final String formula, - @NotNull final String columnParamName, - @NotNull final SelectColumn[] groupByColumns) { - return applyToAllBy(AggregationControl.DEFAULT_FOR_OPERATOR, inputTable, formula, columnParamName, - groupByColumns); - } - - public static QueryTable applyToAllBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final String formula, - @NotNull final String columnParamName, - @NotNull final String... groupByColumnNames) { - return applyToAllBy(aggregationControl, inputTable, formula, columnParamName, - SelectColumnFactory.getExpressions(groupByColumnNames)); - } - - public static QueryTable applyToAllBy(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final String formula, - @NotNull final String columnParamName, - @NotNull final SelectColumn[] groupByColumns) { - return ChunkedOperatorAggregationHelper.aggregation(aggregationControl, - new FormulaAggregationFactory(formula, columnParamName), inputTable, groupByColumns); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.java deleted file mode 100644 index 54c378ee33c..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.java +++ /dev/null @@ -1,70 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.impl.select.MatchPairFactory; -import io.deephaven.engine.table.impl.select.SelectColumnFactory; -import io.deephaven.engine.table.impl.QueryTable; -import io.deephaven.engine.table.impl.select.SelectColumn; -import io.deephaven.engine.table.ChunkSource; -import org.jetbrains.annotations.NotNull; - -import java.util.Arrays; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * An {@link AggregationContextFactory} used in the implementation of {@link Table#groupBy}. - */ -public class GroupByAggregationFactory implements AggregationContextFactory { - - private static final GroupByAggregationFactory INSTANCE = new GroupByAggregationFactory(); - - public static AggregationContextFactory getInstance() { - return INSTANCE; - } - - private GroupByAggregationFactory() {} - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table inputTable, - @NotNull final String... groupByColumnNames) { - final Set groupByColumnNameSet = Arrays.stream(groupByColumnNames).collect(Collectors.toSet()); - final String[] resultColumnNames = inputTable.getDefinition().getColumnNames().stream() - .filter(cn -> !groupByColumnNameSet.contains(cn)).toArray(String[]::new); - // noinspection unchecked - return new AggregationContext( - new IterativeChunkedAggregationOperator[] {new GroupByChunkedOperator((QueryTable) inputTable, true, - MatchPairFactory.getExpressions(resultColumnNames))}, - new String[][] {CollectionUtil.ZERO_LENGTH_STRING_ARRAY}, - new ChunkSource.WithPrev[] {null}); - } - - @Override - public String toString() { - return "By"; - } - - public static QueryTable by(@NotNull final QueryTable inputTable, - @NotNull final String... groupByColumnNames) { - return by(AggregationControl.DEFAULT_FOR_OPERATOR, inputTable, groupByColumnNames); - } - - public static QueryTable by(@NotNull final QueryTable inputTable, - @NotNull final SelectColumn[] groupByColumns) { - return by(AggregationControl.DEFAULT_FOR_OPERATOR, inputTable, groupByColumns); - } - - public static QueryTable by(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final String... groupByColumnNames) { - return by(aggregationControl, inputTable, SelectColumnFactory.getExpressions(groupByColumnNames)); - } - - public static QueryTable by(@NotNull final AggregationControl aggregationControl, - @NotNull final QueryTable inputTable, - @NotNull final SelectColumn[] groupByColumns) { - return ChunkedOperatorAggregationHelper.aggregation(aggregationControl, getInstance(), inputTable, - groupByColumns); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java deleted file mode 100644 index 5bbc9bafe08..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.java +++ /dev/null @@ -1,21 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.datastructures.util.CollectionUtil; -import io.deephaven.engine.table.Table; -import io.deephaven.engine.table.ChunkSource; -import org.jetbrains.annotations.NotNull; - -/** - * Simplest factory that has no operators or results, used to implement selectDistinct. - */ -public class KeyOnlyAggregationFactory implements AggregationContextFactory { - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - // noinspection unchecked - return new AggregationContext( - IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY, - CollectionUtil.ZERO_LENGTH_STRING_ARRAY_ARRAY, - ChunkSource.WithPrev.ZERO_LENGTH_CHUNK_SOURCE_WITH_PREV_ARRAY); - } -} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.java deleted file mode 100644 index 060c1bbe5da..00000000000 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.java +++ /dev/null @@ -1,176 +0,0 @@ -package io.deephaven.engine.table.impl.by; - -import io.deephaven.chunk.ChunkType; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.table.*; -import org.apache.commons.lang3.mutable.MutableBoolean; -import org.jetbrains.annotations.NotNull; - -import java.util.*; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -public class WeightedAverageSumAggregationFactory implements AggregationContextFactory { - private final String weightName; - private final boolean isSum; - - public WeightedAverageSumAggregationFactory(final String weightName, boolean isSum) { - this.weightName = weightName; - this.isSum = isSum; - } - - @Override - public AggregationContext makeAggregationContext(@NotNull final Table table, - @NotNull final String... groupByColumns) { - return getAllColumnOperators(table, groupByColumns); - } - - private AggregationContext getAllColumnOperators(Table withView, String[] groupByNameArray) { - final Set groupByNames = new HashSet<>(Arrays.asList(groupByNameArray)); - final int operatorColumnCount = withView.getColumnSourceMap().size() - groupByNames.size() - 1; - - final List operators = new ArrayList<>(operatorColumnCount + 1); - final List inputNames = new ArrayList<>(operatorColumnCount); - final List> inputColumns = new ArrayList<>(operatorColumnCount + 1); - - final MatchPair[] resultPairs = withView.getDefinition().getColumnStream() - .map(ColumnDefinition::getName) - .filter(cn -> !cn.equals(weightName) && !groupByNames.contains(cn)) - .map(cn -> new MatchPair(cn, cn)) - .toArray(MatchPair[]::new); - getOperatorsAndInputs(withView, weightName, isSum, resultPairs, operators, inputNames, inputColumns); - - // noinspection unchecked - return new AggregationContext( - operators.toArray(IterativeChunkedAggregationOperator[]::new), - inputNames.toArray(String[][]::new), - inputColumns.toArray(ChunkSource.WithPrev[]::new)); - } - - private static boolean isFloatingPoint(@NotNull final ChunkType chunkType) { - return chunkType == ChunkType.Float - || chunkType == ChunkType.Double; - } - - private static boolean isInteger(@NotNull final ChunkType chunkType) { - return chunkType == ChunkType.Char - || chunkType == ChunkType.Byte - || chunkType == ChunkType.Short - || chunkType == ChunkType.Int - || chunkType == ChunkType.Long; - } - - private enum ResultType { - INTEGER, FLOATING_POINT - } - - private static class Result { - private final MatchPair pair; - private final ResultType type; - private final ColumnSource source; - - private Result(MatchPair pair, ResultType type, ColumnSource source) { - this.pair = pair; - this.type = type; - this.source = source; - } - } - - static void getOperatorsAndInputs(Table withView, - String weightName, - boolean isSum, - MatchPair[] resultPairs, - List resultOperators, - List resultInputNames, - List> resultInputColumns) { - final ColumnSource weightSource = withView.getColumnSource(weightName); - final boolean weightSourceIsFloatingPoint; - if (isInteger(weightSource.getChunkType())) { - weightSourceIsFloatingPoint = false; - } else if (isFloatingPoint(weightSource.getChunkType())) { - weightSourceIsFloatingPoint = true; - } else { - throw new UnsupportedOperationException(String.format("Invalid type %s in weight column %s for %s", - weightSource.getType(), weightName, toString(isSum, weightName))); - } - - final MutableBoolean anyIntegerResults = new MutableBoolean(); - final MutableBoolean anyFloatingPointResults = new MutableBoolean(); - final List results = Arrays.stream(resultPairs).map(pair -> { - final ColumnSource inputSource = withView.getColumnSource(pair.rightColumn); - final ResultType resultType; - if (isInteger(inputSource.getChunkType())) { - if (!weightSourceIsFloatingPoint && isSum) { - anyIntegerResults.setTrue(); - resultType = ResultType.INTEGER; - } else { - anyFloatingPointResults.setTrue(); - resultType = ResultType.FLOATING_POINT; - } - } else if (isFloatingPoint(inputSource.getChunkType())) { - anyFloatingPointResults.setTrue(); - resultType = ResultType.FLOATING_POINT; - } else { - throw new UnsupportedOperationException(String.format("Invalid type %s in column %s for %s", - inputSource.getType(), pair.rightColumn, toString(isSum, weightName))); - } - return new Result(pair, resultType, inputSource); - }).collect(Collectors.toList()); - - final LongWeightRecordingInternalOperator longWeightOperator; - if (anyIntegerResults.booleanValue()) { - longWeightOperator = new LongWeightRecordingInternalOperator(weightSource.getChunkType()); - resultOperators.add(longWeightOperator); - resultInputNames.add(Stream.concat( - Stream.of(weightName), - results.stream() - .filter(r -> r.type == ResultType.INTEGER).map(r -> r.pair.rightColumn)) - .toArray(String[]::new)); - // noinspection unchecked - resultInputColumns.add(weightSource); - } else { - longWeightOperator = null; - } - - final DoubleWeightRecordingInternalOperator doubleWeightOperator; - if (anyFloatingPointResults.booleanValue()) { - doubleWeightOperator = new DoubleWeightRecordingInternalOperator(weightSource.getChunkType()); - resultOperators.add(doubleWeightOperator); - resultInputNames.add(Stream.concat( - Stream.of(weightName), - results.stream() - .filter(r -> r.type == ResultType.FLOATING_POINT).map(r -> r.pair.rightColumn)) - .toArray(String[]::new)); - // noinspection unchecked - resultInputColumns.add(weightSource); - } else { - doubleWeightOperator = null; - } - - results.forEach(r -> { - if (isSum) { - if (r.type == ResultType.INTEGER) { - resultOperators.add(new LongChunkedWeightedSumOperator( - r.source.getChunkType(), longWeightOperator, r.pair.leftColumn)); - } else { - resultOperators.add(new DoubleChunkedWeightedSumOperator( - r.source.getChunkType(), doubleWeightOperator, r.pair.leftColumn)); - } - } else { - resultOperators.add(new ChunkedWeightedAverageOperator( - r.source.getChunkType(), doubleWeightOperator, r.pair.leftColumn)); - } - resultInputNames.add(new String[] {r.pair.rightColumn, weightName}); - resultInputColumns.add(r.source); - }); - } - - @Override - public String toString() { - return toString(isSum, weightName); - } - - private static String toString(final boolean isSum, final String weightName) { - return "Weighted" + (isSum ? "Sum" : "Avg") + "(" + weightName + ")"; - } -} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index ba41b9ee61c..f509948dd21 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -1,13 +1,16 @@ package io.deephaven.engine.table.impl; +import io.deephaven.api.ColumnName; import io.deephaven.api.Selectable; import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.datastructures.util.CollectionUtil; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.RowSetFactory; import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.table.*; import io.deephaven.engine.table.impl.indexer.RowSetIndexer; +import io.deephaven.qst.table.AggregateAllByTable; import io.deephaven.time.DateTimeUtils; import io.deephaven.engine.util.TableDiff; import io.deephaven.engine.util.TableTools; @@ -74,6 +77,14 @@ public void tearDown() throws Exception { // region Static chunked groupBy() tests + private static AggregationContextFactory makeGroupByACF( + @NotNull final Table table, @NotNull final String... groupByColumns) { + return AggregationProcessor.forAggregation(List.of( + AggregateAllByTable.singleAggregation(AggSpec.group(), Selectable.from(groupByColumns), + table.getDefinition().getColumnStream().map(ColumnDefinition::getName) + .map(ColumnName::of).collect(Collectors.toList())).orElseThrow())); + } + private static Table individualStaticByTest(@NotNull final Table input, @Nullable final AggregationControl aggregationControl, @NotNull final String... keyColumns) { final Table adjustedInput = input.update("__Pre_Agg_II__=ii"); @@ -109,8 +120,9 @@ private static Table individualStaticByTest(@NotNull final Table input, final SelectColumn[] keySelectColumns = SelectColumnFactory.getExpressions(keyColumns); final String[] keyNames = Arrays.stream(keySelectColumns).map(SelectColumn::getName).distinct().toArray(String[]::new); - final Table aggregatedInput = GroupByAggregationFactory.by( + final Table aggregatedInput = ChunkedOperatorAggregationHelper.aggregation( aggregationControl == null ? AggregationControl.DEFAULT : aggregationControl, + makeGroupByACF(adjustedInput, keyColumns), (QueryTable) adjustedInput, keySelectColumns); actualKeys = keyNames.length == 0 ? aggregatedInput.dropColumns(aggregatedInput.getDefinition().getColumnNamesArray()) @@ -198,6 +210,8 @@ private static class IncrementalFirstStaticAfterByResultSupplier implements Supp private final QueryTable input; private final String[] columns; + private final AggregationContextFactory acf; + private final AtomicBoolean firstTime = new AtomicBoolean(true); private IncrementalFirstStaticAfterByResultSupplier(@NotNull final AggregationControl control, @@ -205,6 +219,7 @@ private IncrementalFirstStaticAfterByResultSupplier(@NotNull final AggregationCo this.control = control; this.input = input; this.columns = columns; + acf = makeGroupByACF(input, columns); } /** @@ -221,9 +236,11 @@ public final Table get() { final String[] keyNames = Arrays.stream(keySelectColumns).map(SelectColumn::getName).distinct().toArray(String[]::new); if (firstTime.compareAndSet(true, false)) { - return GroupByAggregationFactory.by(control, input, keySelectColumns).sort(keyNames); + return ChunkedOperatorAggregationHelper + .aggregation(control, acf, input, keySelectColumns).sort(keyNames); } - return GroupByAggregationFactory.by(control, (QueryTable) input.silent(), keySelectColumns).sort(keyNames); + return ChunkedOperatorAggregationHelper + .aggregation(control, acf, (QueryTable) input.silent(), keySelectColumns).sort(keyNames); } } @@ -305,7 +322,7 @@ public boolean shouldProbeShift(long shiftSize, int numStates) { new EvalNugget() { @Override protected final Table e() { - return GroupByAggregationFactory.by(merged, "StrCol").update("IntColSum=sum(IntCol)"); + return merged.groupBy("StrCol").update("IntColSum=sum(IntCol)"); } } }; @@ -381,7 +398,7 @@ public void testIncrementalNoKeyBy() { new EvalNugget() { @Override protected Table e() { - return GroupByAggregationFactory.by(input1).update("IntColSum=sum(IntCol)"); + return input1.groupBy().update("IntColSum=sum(IntCol)"); } } }; @@ -3453,9 +3470,8 @@ public void testIds6321() { final FuzzerPrintListener soucePrinter = new FuzzerPrintListener("source", source); source.listenForUpdates(soucePrinter); - final QueryTable exposedLastBy = ChunkedOperatorAggregationHelper.aggregation( - new FirstOrLastByAggregationFactory(false, "ExposedRowRedirection"), source, - SelectColumnFactory.getExpressions("Key")); + final QueryTable exposedLastBy = (QueryTable) source.aggBy(Aggregation.AggLastRowKey("ExposedRowRedirection"), + "Key"); final TableUpdateValidator validator = TableUpdateValidator.make(exposedLastBy); final QueryTable validatorResult = validator.getResultTable(); final FailureListener validatorListener = new FailureListener(); From bc20d8786904083d66be0d5a75bc1e10ac494f99 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Sat, 29 Jan 2022 12:01:21 -0500 Subject: [PATCH 31/44] replication + spotless + generation --- .../io/deephaven/api/agg/spec/AggSpec.json | 1 + .../api/agg/spec/AggSpec/Visitor.json | 2 +- .../api/agg/spec/AggSpecColumnReferences.json | 2 +- .../deephaven/api/agg/spec/AggSpecFreeze.json | 11 ++++ .../impl/ApplicationService/Listener.json | 2 +- .../doc/io/deephaven/client/impl/Export.json | 2 +- .../io/deephaven/client/impl/ExportId.json | 3 +- .../deephaven/client/impl/FetchedObject.json | 13 +++++ .../deephaven/client/impl/FieldChanges.json | 12 +++++ .../io/deephaven/client/impl/FieldInfo.json | 14 +++++ .../deephaven/client/impl/ObjectService.json | 8 +++ .../io/deephaven/client/impl/SessionImpl.json | 1 + .../deephaven/client/impl/script/Changes.json | 6 +-- .../client/impl/script/Changes/Builder.json | 7 +-- .../impl/script/VariableDefinition.json | 10 ---- .../external/FastDoubleParser.json | 9 ---- .../FastDoubleParserFromByteArray.json | 9 ---- .../FastDoubleParserFromCharArray.json | 9 ---- .../deephaven/engine/table/ColumnSource.json | 2 + .../WritableSourceWithEnsurePrevious.json | 10 ++++ .../engine/table/impl/HierarchicalTable.json | 2 +- .../engine/table/impl/PrevColumnSource.json | 4 +- .../engine/table/impl/QueryTable.json | 2 +- .../impl/UnboxedDateTimeColumnSource.json | 4 +- .../table/impl/by/AggregationHelper.json | 10 ---- .../by/FirstOrLastByAggregationFactory.json | 9 ---- .../impl/by/FormulaAggregationFactory.json | 11 ---- .../impl/by/FreezeByAggregationFactory.json | 9 ---- .../impl/by/GroupByAggregationFactory.json | 12 ----- ...entalChunkedByAggregationStateManager.json | 9 ---- .../impl/by/KeyOnlyAggregationFactory.json | 9 ---- ...taticChunkedByAggregationStateManager.json | 9 ---- .../WeightedAverageSumAggregationFactory.json | 9 ---- .../impl/lang/QueryLanguageFunctionUtils.json | 20 ++++---- .../impl/select/AbstractFormulaColumn.json | 1 + .../table/impl/select/DhFormulaColumn.json | 4 +- .../table/impl/select/FunctionalColumn.json | 1 + .../select/MultiSourceFunctionalColumn.json | 1 + .../table/impl/select/NullSelectColumn.json | 1 + .../impl/select/ReinterpretedColumn.json | 1 + .../table/impl/select/SelectColumn.json | 1 + .../table/impl/select/SourceColumn.json | 1 + .../table/impl/select/SwitchColumn.json | 1 + .../impl/select/analyzers/BaseLayer.json | 1 + .../select/analyzers/PreserveColumnLayer.json | 1 + .../select/analyzers/RedirectionLayer.json | 1 + .../analyzers/SelectAndViewAnalyzer.json | 1 + .../ImmediateJobScheduler.json | 3 +- .../SelectAndViewAnalyzer/JobScheduler.json | 3 +- ...rationInitializationPoolJobScheduler.json} | 7 +-- .../UpdateGraphProcessorJobScheduler.json | 3 +- .../select/analyzers/SelectColumnLayer.json | 1 + .../select/analyzers/ViewColumnLayer.json | 1 + .../select/python/FormulaColumnPython.json | 4 +- .../AbstractSparseLongArraySource.json | 1 + .../impl/sources/BitMaskingColumnSource.json | 2 + .../impl/sources/BitShiftingColumnSource.json | 2 + .../sources/BooleanAsByteColumnSource.json | 4 +- .../sources/BooleanSparseArraySource.json | 1 + .../table/impl/sources/BoxedColumnSource.json | 4 +- .../sources/ByteAsBooleanColumnSource.json | 2 + .../impl/sources/ByteSparseArraySource.json | 1 + .../sources/CharacterSparseArraySource.json | 1 + .../sources/CrossJoinRightColumnSource.json | 2 + .../sources/DateTimeAsLongColumnSource.json | 4 +- .../impl/sources/DelegatingColumnSource.json | 2 + .../impl/sources/DoubleSparseArraySource.json | 1 + .../impl/sources/FloatSparseArraySource.json | 1 + .../sources/IntegerSparseArraySource.json | 1 + .../sources/LongAsDateTimeColumnSource.json | 2 + .../impl/sources/ObjectSparseArraySource.json | 1 + .../impl/sources/RedirectedColumnSource.json | 2 + .../impl/sources/ReversedColumnSource.json | 2 + .../impl/sources/ShortSparseArraySource.json | 1 + .../impl/sources/SwitchColumnSource.json | 2 + .../UngroupedByteArrayColumnSource.json | 4 +- .../UngroupedByteVectorColumnSource.json | 4 +- .../UngroupedCharArrayColumnSource.json | 4 +- .../UngroupedCharVectorColumnSource.json | 4 +- .../UngroupedDoubleArrayColumnSource.json | 4 +- .../UngroupedDoubleVectorColumnSource.json | 4 +- .../UngroupedFloatArrayColumnSource.json | 4 +- .../UngroupedFloatVectorColumnSource.json | 4 +- .../UngroupedIntArrayColumnSource.json | 4 +- .../UngroupedIntVectorColumnSource.json | 4 +- .../UngroupedLongArrayColumnSource.json | 4 +- .../UngroupedLongVectorColumnSource.json | 4 +- .../UngroupedShortArrayColumnSource.json | 4 +- .../UngroupedShortVectorColumnSource.json | 4 +- .../table/impl/sources/UnionColumnSource.json | 4 +- .../table/impl/sources/ViewColumnSource.json | 2 + .../aggregate/ByteAggregateColumnSource.json | 2 + .../aggregate/CharAggregateColumnSource.json | 2 + .../DoubleAggregateColumnSource.json | 2 + .../aggregate/FloatAggregateColumnSource.json | 2 + .../aggregate/IntAggregateColumnSource.json | 2 + .../aggregate/LongAggregateColumnSource.json | 2 + .../ObjectAggregateColumnSource.json | 2 + .../aggregate/ShortAggregateColumnSource.json | 2 + .../immutable/Immutable2DByteArraySource.json | 1 + .../immutable/Immutable2DCharArraySource.json | 1 + .../Immutable2DDoubleArraySource.json | 1 + .../Immutable2DFloatArraySource.json | 1 + .../immutable/Immutable2DIntArraySource.json | 1 + .../immutable/Immutable2DLongArraySource.json | 1 + .../Immutable2DObjectArraySource.json | 1 + .../Immutable2DShortArraySource.json | 1 + .../immutable/ImmutableByteArraySource.json | 1 + .../immutable/ImmutableCharArraySource.json | 1 + .../immutable/ImmutableDoubleArraySource.json | 1 + .../immutable/ImmutableFloatArraySource.json | 1 + .../immutable/ImmutableIntArraySource.json | 1 + .../immutable/ImmutableLongArraySource.json | 1 + .../immutable/ImmutableObjectArraySource.json | 1 + .../immutable/ImmutableShortArraySource.json | 1 + .../AppendOnlyArrayBackedMutableTable.json | 2 + .../util/KeyedArrayBackedMutableTable.json | 2 + .../SortedMergeColumnSource.json | 4 +- .../engine/util/BigDecimalUtils.json | 11 ++++ .../BigDecimalUtils/PrecisionAndScale.json | 7 +++ .../util/BigDecimalUtils/PropertyNames.json | 7 +++ .../engine/util/ExportedObjectType.json | 13 ----- .../GroovySnapshot.json | 6 +++ .../io/deephaven/engine/util/IsWidget.json | 14 ----- .../PythonSnapshot.json | 6 +++ .../FigureWidgetPluginRegistration.json | 9 ++++ .../figure/FigureWidgetTranslator.json | 8 +++ .../figure/FigureWidgetTypePlugin.json | 10 ++++ .../deephaven/kafka/KafkaTools/Produce.json | 4 +- .../doc/io/deephaven/plugin/Plugin.json | 9 ++++ .../io/deephaven/plugin/Plugin/Visitor.json | 8 +++ .../doc/io/deephaven/plugin/Registration.json | 9 ++++ .../plugin/Registration/Callback.json | 8 +++ .../io/deephaven/plugin/type/ObjectType.json | 11 ++++ .../plugin/type/ObjectType/Exporter.json | 9 ++++ .../type/ObjectType/Exporter/Reference.json | 10 ++++ .../deephaven/plugin/type/ObjectTypeBase.json | 10 ++++ .../plugin/type/ObjectTypeClassBase.json | 14 +++++ .../plugin/type/ObjectTypeLookup.json | 9 ++++ .../plugin/type/ObjectTypeLookup/NoOp.json | 10 ++++ .../plugin/type/ObjectTypeRegistration.json | 9 ++++ .../io/deephaven/proto/DeephavenChannel.json | 3 ++ .../proto/backplane/grpc/CustomInfo.json | 26 ---------- .../backplane/grpc/CustomInfo/Builder.json | 29 ----------- .../backplane/grpc/CustomInfoOrBuilder.json | 9 ---- .../backplane/grpc/FetchObjectRequest.json | 27 ++++++++++ .../grpc/FetchObjectRequest/Builder.json | 31 +++++++++++ .../grpc/FetchObjectRequestOrBuilder.json | 10 ++++ .../backplane/grpc/FetchObjectResponse.json | 32 ++++++++++++ .../grpc/FetchObjectResponse/Builder.json | 45 ++++++++++++++++ .../grpc/FetchObjectResponseOrBuilder.json | 15 ++++++ .../proto/backplane/grpc/FieldInfo.json | 21 ++++---- .../backplane/grpc/FieldInfo/Builder.json | 51 ++++++++----------- .../backplane/grpc/FieldInfo/FieldType.json | 34 ------------- .../grpc/FieldInfo/FieldType/Builder.json | 46 ----------------- .../grpc/FieldInfo/FieldType/FieldCase.json | 11 ---- .../grpc/FieldInfo/FieldTypeOrBuilder.json | 17 ------- .../backplane/grpc/FieldInfoOrBuilder.json | 21 ++++---- .../proto/backplane/grpc/FigureInfo.json | 24 --------- .../backplane/grpc/FigureInfo/Builder.json | 24 --------- .../backplane/grpc/FigureInfoOrBuilder.json | 6 --- .../proto/backplane/grpc/Object.json | 9 ++++ .../backplane/grpc/ObjectServiceGrpc.json | 12 +++++ .../ObjectServiceBlockingStub.json | 8 +++ .../ObjectServiceFutureStub.json | 8 +++ .../ObjectServiceImplBase.json | 9 ++++ .../ObjectServiceGrpc/ObjectServiceStub.json | 8 +++ .../proto/backplane/grpc/RemovedField.json | 24 --------- .../backplane/grpc/RemovedField/Builder.json | 24 --------- .../backplane/grpc/RemovedFieldOrBuilder.json | 6 --- .../proto/backplane/grpc/TableInfo.json | 27 ---------- .../backplane/grpc/TableInfo/Builder.json | 33 ------------ .../backplane/grpc/TableInfoOrBuilder.json | 10 ---- .../proto/backplane/grpc/TypedTicket.json | 29 +++++++++++ .../backplane/grpc/TypedTicket/Builder.json | 36 +++++++++++++ .../backplane/grpc/TypedTicketOrBuilder.json | 12 +++++ .../script/grpc/ConsoleServiceGrpc.json | 1 - .../ConsoleServiceBlockingStub.json | 1 - .../ConsoleServiceFutureStub.json | 1 - .../ConsoleServiceImplBase.json | 1 - .../ConsoleServiceStub.json | 1 - .../script/grpc/ExecuteCommandResponse.json | 18 ++----- .../grpc/ExecuteCommandResponse/Builder.json | 48 +++-------------- .../grpc/ExecuteCommandResponseOrBuilder.json | 18 ++----- .../script/grpc/FetchFigureRequest.json | 27 ---------- .../grpc/FetchFigureRequest/Builder.json | 31 ----------- .../grpc/FetchFigureRequestOrBuilder.json | 10 ---- .../script/grpc/FetchFigureResponse.json | 27 ---------- .../grpc/FetchFigureResponse/Builder.json | 31 ----------- .../grpc/FetchFigureResponseOrBuilder.json | 10 ---- .../script/grpc/FigureDescriptor.json | 5 -- .../script/grpc/FigureDescriptor/Builder.json | 13 ----- .../grpc/FigureDescriptorOrBuilder.json | 5 -- .../script/grpc/VariableDefinition.json | 31 ----------- .../grpc/VariableDefinition/Builder.json | 41 --------------- .../grpc/VariableDefinitionOrBuilder.json | 14 ----- .../appmode/ScriptApplicationState.json | 1 - ...dexGenerator.json => RowSetGenerator.json} | 4 +- .../console/ConsoleServiceGrpcImpl.json | 2 +- .../figure/FigureWidgetTranslator.json | 8 --- .../server/object/ObjectServiceGrpcImpl.json | 8 +++ .../server/object/ObjectServiceModule.json | 8 +++ .../deephaven/server/object/TypeLookup.json | 8 +++ .../server/plugin/PluginsAutoDiscovery.json | 9 ++++ .../server/plugin/PluginsModule.json | 9 ++++ .../server/plugin/java/JavaServiceLoader.json | 8 +++ .../plugin/python/PythonModuleLoader.json | 8 +++ .../server/plugin/type/ObjectTypes.json | 10 ++++ .../server/plugin/type/ObjectTypesModule.json | 10 ++++ .../server/table/TableServiceGrpcImpl.json | 1 - .../table/impl/QueryTableAggregationTest.java | 3 +- 211 files changed, 864 insertions(+), 946 deletions(-) create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/client/impl/FetchedObject.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldChanges.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldInfo.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/client/impl/ObjectService.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/client/impl/script/VariableDefinition.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParser.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromByteArray.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromCharArray.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/WritableSourceWithEnsurePrevious.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationHelper.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.json rename Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/{TableMapTransformJobScheduler.json => OperationInitializationPoolJobScheduler.json} (69%) create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PrecisionAndScale.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PropertyNames.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/ExportedObjectType.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/GroovyDeephavenSession/GroovySnapshot.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/IsWidget.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/util/PythonDeephavenSession/PythonSnapshot.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetPluginRegistration.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTranslator.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTypePlugin.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin/Visitor.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/Registration.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/Registration/Callback.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter/Reference.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeBase.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeClassBase.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup/NoOp.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeRegistration.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfoOrBuilder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest/Builder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequestOrBuilder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse/Builder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponseOrBuilder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/FieldCase.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldTypeOrBuilder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfoOrBuilder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/Object.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceBlockingStub.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceFutureStub.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceImplBase.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceStub.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedFieldOrBuilder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfoOrBuilder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket/Builder.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicketOrBuilder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequestOrBuilder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponseOrBuilder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition/Builder.json delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinitionOrBuilder.json rename Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/{IndexGenerator.json => RowSetGenerator.json} (68%) delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/console/figure/FigureWidgetTranslator.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceGrpcImpl.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceModule.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/object/TypeLookup.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsAutoDiscovery.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsModule.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/plugin/java/JavaServiceLoader.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/plugin/python/PythonModuleLoader.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypes.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypesModule.json diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json index bbcb1f094e8..d61af47afb5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json @@ -10,6 +10,7 @@ "distinct": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecDistinct", "first": ":return: io.deephaven.api.agg.spec.AggSpecFirst", "formula": "*Overload 1* \n :param formula: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 2* \n :param formula: java.lang.String\n :param formulaParam: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula", + "freeze": ":return: io.deephaven.api.agg.spec.AggSpecFreeze", "group": ":return: io.deephaven.api.agg.spec.AggSpecGroup", "last": ":return: io.deephaven.api.agg.spec.AggSpecLast", "max": ":return: io.deephaven.api.agg.spec.AggSpecMax", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json index 8ac5667afe0..488ab2fca2e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpec$Visitor", "methods": { - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 6* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 7* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 8* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 9* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 10* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 11* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 12* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 13* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 14* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 15* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 16* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 17* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 18* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 19* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 20* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 21* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 22* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" + "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param freeze: io.deephaven.api.agg.spec.AggSpecFreeze\n \n*Overload 6* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 7* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 8* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 9* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 10* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 11* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 12* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 13* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 14* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 15* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 16* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 17* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 18* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 19* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 20* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 21* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 22* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 23* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" }, "path": "io.deephaven.api.agg.spec.AggSpec.Visitor", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json index 3df76d3ae3b..f65fcd86e09 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json @@ -3,7 +3,7 @@ "methods": { "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:return: java.util.Set", "out": ":return: java.util.Set", - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 6* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 7* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 8* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 9* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 10* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 11* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 12* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 13* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 14* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 15* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 16* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 17* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 18* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 19* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 20* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 21* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 22* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" + "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param freeze: io.deephaven.api.agg.spec.AggSpecFreeze\n \n*Overload 6* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 7* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 8* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 9* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 10* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 11* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 12* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 13* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 14* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 15* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 16* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 17* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 18* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 19* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 20* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 21* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 22* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 23* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" }, "path": "io.deephaven.api.agg.spec.AggSpecColumnReferences", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json new file mode 100644 index 00000000000..1605da95d4d --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.api.agg.spec.AggSpecFreeze", + "methods": { + "description": ":return: java.lang.String", + "of": ":return: io.deephaven.api.agg.spec.AggSpecFreeze", + "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + }, + "path": "io.deephaven.api.agg.spec.AggSpecFreeze", + "text": "Aggregation that freezes the first value for each group and ignores subsequent changes. When groups are removed, the\n corresponding output row is removed. When groups are re-added (on a subsequent update cycle), the newly added value\n is then frozen.\n \n Only one row per group is allowed in the output, because the operation has no way to determine which row to freeze\n otherwise. This is a constraint on the input data.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/ApplicationService/Listener.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/ApplicationService/Listener.json index 9443bad73e1..06e3beaecde 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/ApplicationService/Listener.json +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/ApplicationService/Listener.json @@ -2,7 +2,7 @@ "className": "io.deephaven.client.impl.ApplicationService$Listener", "methods": { "onError": ":param t: java.lang.Throwable", - "onNext": ":param fields: io.deephaven.proto.backplane.grpc.FieldsChangeUpdate" + "onNext": ":param fields: io.deephaven.client.impl.FieldChanges" }, "path": "io.deephaven.client.impl.ApplicationService.Listener", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/Export.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/Export.json index d75373667d9..11f75410a1c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/Export.json +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/Export.json @@ -13,6 +13,6 @@ "toString": ":return: java.lang.String" }, "path": "io.deephaven.client.impl.Export", - "text": "An export represents a server-side object that is being kept alive.\n\n \n Callers must maintain ownership of their exports, and close them when no longer needed.", + "text": "An export represents a server-side Table that is being kept alive.\n\n \n Callers must maintain ownership of their exports, and close them when no longer needed.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/ExportId.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/ExportId.json index fae71f95508..ec3e859e4a0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/ExportId.json +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/ExportId.json @@ -4,7 +4,8 @@ "exportId": "Get the export ID.\n\n:return: (io.deephaven.client.impl.ExportId) the export ID", "pathId": "Get the path ID.\n\n:return: (io.deephaven.client.impl.PathId) the path ID", "ticketId": "Get the ticket ID.\n\n:return: (io.deephaven.client.impl.TicketId) the ticket ID", - "toString": ":return: java.lang.String" + "toString": ":return: java.lang.String", + "type": ":return: java.util.Optional" }, "path": "io.deephaven.client.impl.ExportId", "text": "An opaque holder for a session export ID.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/FetchedObject.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/FetchedObject.json new file mode 100644 index 00000000000..0954f220f6d --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/FetchedObject.json @@ -0,0 +1,13 @@ +{ + "className": "io.deephaven.client.impl.FetchedObject", + "methods": { + "exportIds": ":return: java.util.List", + "size": ":return: int", + "toByteArray": ":return: byte[]", + "toString": ":return: java.lang.String", + "type": ":return: java.lang.String", + "writeTo": ":param out: java.io.OutputStream" + }, + "path": "io.deephaven.client.impl.FetchedObject", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldChanges.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldChanges.json new file mode 100644 index 00000000000..aeedf7b4ef6 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldChanges.json @@ -0,0 +1,12 @@ +{ + "className": "io.deephaven.client.impl.FieldChanges", + "methods": { + "created": ":return: java.util.List", + "isEmpty": ":return: boolean", + "removed": ":return: java.util.List", + "toString": ":return: java.lang.String", + "updated": ":return: java.util.List" + }, + "path": "io.deephaven.client.impl.FieldChanges", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldInfo.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldInfo.json new file mode 100644 index 00000000000..afd0cc07fff --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/FieldInfo.json @@ -0,0 +1,14 @@ +{ + "className": "io.deephaven.client.impl.FieldInfo", + "methods": { + "applicationId": ":return: java.lang.String", + "applicationName": ":return: java.lang.String", + "description": ":return: java.util.Optional", + "name": ":return: java.lang.String", + "ticket": ":return: io.deephaven.client.impl.TicketId", + "toString": ":return: java.lang.String", + "type": ":return: java.util.Optional" + }, + "path": "io.deephaven.client.impl.FieldInfo", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/ObjectService.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/ObjectService.json new file mode 100644 index 00000000000..ef82942bebe --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/ObjectService.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.client.impl.ObjectService", + "methods": { + "fetchObject": "Fetch the object.\n\n:param type: java.lang.String\n:param ticket: (io.deephaven.client.impl.HasTicketId) - the ticket\n:return: (java.util.concurrent.CompletableFuture) the future" + }, + "path": "io.deephaven.client.impl.ObjectService", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/SessionImpl.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/SessionImpl.json index 4c805bb8ec3..13973fa9cc6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/SessionImpl.json +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/SessionImpl.json @@ -12,6 +12,7 @@ "createFuture": ":param config: io.deephaven.client.impl.SessionImplConfig\n:return: java.util.concurrent.CompletableFuture", "deleteFromInputTable": "Delete source from the input table destination.\n\n:param destination: (io.deephaven.client.impl.HasTicketId) - the input table\n:param source: (io.deephaven.client.impl.HasTicketId) - the source to delete\n:return: (java.util.concurrent.CompletableFuture) the future", "export": "Creates new exports according to the request.\n\n:param request: (io.deephaven.client.impl.ExportsRequest) - the request\n:return: (java.util.List) the exports", + "fetchObject": "Fetch the object.\n\n:param type: java.lang.String\n:param ticketId: io.deephaven.client.impl.HasTicketId\n:return: (java.util.concurrent.CompletableFuture) the future", "newExportId": "Advanced usage, creates a new export ID for this session, but must be managed by the caller. Useful for\n more advanced integrations, particularly around doPut. Callers are responsible for releasing the export ID if necessary.\n\n:return: (io.deephaven.client.impl.ExportId) the new export ID", "publish": "Publishes ticket into the global scope with name.\n\n:param name: (java.lang.String) - the name, must conform to SourceVersion.isName(CharSequence)\n:param ticketId: (io.deephaven.client.impl.HasTicketId) - the export ID\n:return: (java.util.concurrent.CompletableFuture) the publish completable future", "release": "Releases an export ID.\n\n \n Note: this should only be called in combination with exports returned from Session.newExportId().\n\n:param exportId: (io.deephaven.client.impl.ExportId) - the export ID\n:return: (java.util.concurrent.CompletableFuture) the future", diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes.json index ab06efb6fd7..dc784a399ad 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes.json +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes.json @@ -2,11 +2,9 @@ "className": "io.deephaven.client.impl.script.Changes", "methods": { "builder": ":return: io.deephaven.client.impl.script.Changes.Builder", - "created": ":return: java.util.List", + "changes": ":return: io.deephaven.client.impl.FieldChanges", "errorMessage": ":return: java.util.Optional", - "isEmpty": ":return: boolean", - "removed": ":return: java.util.List", - "updated": ":return: java.util.List" + "isEmpty": ":return: boolean" }, "path": "io.deephaven.client.impl.script.Changes", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes/Builder.json index f7f1ced8a49..51607ef152e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes/Builder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/Changes/Builder.json @@ -1,13 +1,8 @@ { "className": "io.deephaven.client.impl.script.Changes$Builder", "methods": { - "addAllCreated": ":param elements: java.lang.Iterable\n:return: io.deephaven.client.impl.script.Changes.Builder", - "addAllRemoved": ":param elements: java.lang.Iterable\n:return: io.deephaven.client.impl.script.Changes.Builder", - "addAllUpdated": ":param elements: java.lang.Iterable\n:return: io.deephaven.client.impl.script.Changes.Builder", - "addCreated": "*Overload 1* \n :param element: io.deephaven.client.impl.script.VariableDefinition\n :return: io.deephaven.client.impl.script.Changes.Builder\n \n*Overload 2* \n :param elements: io.deephaven.client.impl.script.VariableDefinition...\n :return: io.deephaven.client.impl.script.Changes.Builder", - "addRemoved": "*Overload 1* \n :param element: io.deephaven.client.impl.script.VariableDefinition\n :return: io.deephaven.client.impl.script.Changes.Builder\n \n*Overload 2* \n :param elements: io.deephaven.client.impl.script.VariableDefinition...\n :return: io.deephaven.client.impl.script.Changes.Builder", - "addUpdated": "*Overload 1* \n :param element: io.deephaven.client.impl.script.VariableDefinition\n :return: io.deephaven.client.impl.script.Changes.Builder\n \n*Overload 2* \n :param elements: io.deephaven.client.impl.script.VariableDefinition...\n :return: io.deephaven.client.impl.script.Changes.Builder", "build": ":return: io.deephaven.client.impl.script.Changes", + "changes": ":param changes: io.deephaven.client.impl.FieldChanges\n:return: io.deephaven.client.impl.script.Changes.Builder", "errorMessage": ":param errorMessage: java.lang.String\n:return: io.deephaven.client.impl.script.Changes.Builder" }, "path": "io.deephaven.client.impl.script.Changes.Builder", diff --git a/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/VariableDefinition.json b/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/VariableDefinition.json deleted file mode 100644 index d7330e3d5ab..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/client/impl/script/VariableDefinition.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.client.impl.script.VariableDefinition", - "methods": { - "of": ":param type: java.lang.String\n:param title: java.lang.String\n:return: io.deephaven.client.impl.script.VariableDefinition", - "title": ":return: java.lang.String", - "type": ":return: java.lang.String" - }, - "path": "io.deephaven.client.impl.script.VariableDefinition", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParser.json b/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParser.json deleted file mode 100644 index ae676191995..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParser.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.csv.tokenization.external.FastDoubleParser", - "methods": { - "parseDouble": "**Incompatible overloads text - text from the first overload:**\n\nConvenience method for calling parseDouble(CharSequence, int, int).\n\n*Overload 1* \n :param str: (java.lang.CharSequence) - the string to be parsed\n :return: (double) the parsed double value\n \n*Overload 2* \n :param str: (java.lang.CharSequence) - the string to be parsed\n :param offset: (int) - The index of the first character to parse\n :param length: (int) - The number of characters to parse\n :return: (double) the parsed double value" - }, - "path": "io.deephaven.csv.tokenization.external.FastDoubleParser", - "text": "This is a C++ to Java port of Daniel Lemire's fast_double_parser.\n \n The code has been changed, so that it parses the same syntax as Double.parseDouble(String).\n \n References:\n \nDaniel Lemire, fast_double_parser, 4x faster than strtod. Apache License 2.0 or Boost Software License.\ngithub.com\nDaniel Lemire, fast_float number parsing library: 4x faster than strtod. Apache License 2.0.\ngithub.com\nDaniel Lemire, Number Parsing at a Gigabyte per Second, Software: Practice and Experience 51 (8), 2021.\n arXiv.2101.11408v3 [cs.DS] 24 Feb 2021\narxiv.org", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromByteArray.json b/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromByteArray.json deleted file mode 100644 index 58afbfed046..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromByteArray.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.csv.tokenization.external.FastDoubleParserFromByteArray", - "methods": { - "parseDouble": "**Incompatible overloads text - text from the first overload:**\n\nConvenience method for calling parseDouble(byte[], int, int).\n\n*Overload 1* \n :param str: (byte[]) - the string to be parsed, a byte array with characters in ISO-8859-1, ASCII or UTF-8 encoding\n :return: (double) the parsed double value\n \n*Overload 2* \n :param str: (byte[]) - the string to be parsed, a byte array with characters in ISO-8859-1, ASCII or UTF-8 encoding\n :param off: (int) - The index of the first byte to parse\n :param len: (int) - The number of bytes to parse\n :return: (double) the parsed double value" - }, - "path": "io.deephaven.csv.tokenization.external.FastDoubleParserFromByteArray", - "text": "This is a C++ to Java port of Daniel Lemire's fast_double_parser.\n \n The code has been changed, so that it parses the same syntax as Double.parseDouble(String).\n \n References:\n \nDaniel Lemire, fast_double_parser, 4x faster than strtod. Apache License 2.0 or Boost Software License.\ngithub.com\nDaniel Lemire, fast_float number parsing library: 4x faster than strtod. Apache License 2.0.\ngithub.com\nDaniel Lemire, Number Parsing at a Gigabyte per Second, Software: Practice and Experience 51 (8), 2021.\n arXiv.2101.11408v3 [cs.DS] 24 Feb 2021\narxiv.org", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromCharArray.json b/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromCharArray.json deleted file mode 100644 index 6653712950b..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/csv/tokenization/external/FastDoubleParserFromCharArray.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.csv.tokenization.external.FastDoubleParserFromCharArray", - "methods": { - "parseDouble": "**Incompatible overloads text - text from the first overload:**\n\nConvenience method for calling parseDouble(char[], int, int).\n\n*Overload 1* \n :param str: (char[]) - the string to be parsed\n :return: (double) the parsed double value\n \n*Overload 2* \n :param str: (char[]) - the string to be parsed, a byte array with characters in ISO-8859-1, ASCII or UTF-8 encoding\n :param off: (int) - The index of the first character to parse\n :param len: (int) - The number of characters to parse\n :return: (double) the parsed double value" - }, - "path": "io.deephaven.csv.tokenization.external.FastDoubleParserFromCharArray", - "text": "This is a C++ to Java port of Daniel Lemire's fast_double_parser.\n \n The code has been changed, so that it parses the same syntax as Double.parseDouble(String).\n \n References:\n \nDaniel Lemire, fast_double_parser, 4x faster than strtod. Apache License 2.0 or Boost Software License.\ngithub.com\nDaniel Lemire, fast_float number parsing library: 4x faster than strtod. Apache License 2.0.\ngithub.com\nDaniel Lemire, Number Parsing at a Gigabyte per Second, Software: Practice and Experience 51 (8), 2021.\n arXiv.2101.11408v3 [cs.DS] 24 Feb 2021\narxiv.org", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/ColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/ColumnSource.json index 8d6d14020f3..0bde2f13e4a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/ColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/ColumnSource.json @@ -16,7 +16,9 @@ "getType": ":return: java.lang.Class", "getValuesMapping": ":param subRange: io.deephaven.engine.rowset.RowSet\n:return: java.util.Map", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "match": ":param invertMatch: boolean\n:param usePrev: boolean\n:param caseInsensitive: boolean\n:param mapper: io.deephaven.engine.rowset.RowSet\n:param keys: java.lang.Object...\n:return: io.deephaven.engine.rowset.WritableRowSet", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "reinterpret": "Provide an alternative view into the data underlying this column source.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternative type to expose\n:return: (io.deephaven.engine.table.ColumnSource) A column source of the alternate data type, backed by the same underlying data.", "releaseCachedResources": "Release any resources held for caching purposes. Implementations need not guarantee that concurrent accesses are\n correct, as the purpose of this method is to ensure cleanup for column sources that will no longer be used.", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/WritableSourceWithEnsurePrevious.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/WritableSourceWithEnsurePrevious.json new file mode 100644 index 00000000000..db6becd75d8 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/WritableSourceWithEnsurePrevious.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.engine.table.WritableSourceWithEnsurePrevious", + "methods": { + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", + "providesEnsurePrevious": "Does the specified WritableColumnSource provide the ensurePrevious function?\n\n:param wcs: (io.deephaven.engine.table.WritableColumnSource) - the WritableColumnSource to check\n:return: (boolean) true if ensurePrevious can be called on wcs" + }, + "path": "io.deephaven.engine.table.WritableSourceWithEnsurePrevious", + "text": "A writable source that allows setting all previous values with one operation rather than on each set.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTable.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTable.json index f1a5ae1ac7a..be6fb301238 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTable.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/HierarchicalTable.json @@ -4,7 +4,7 @@ "aggAllBy": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:param groupByColumns: io.deephaven.api.Selectable...\n:return: io.deephaven.engine.table.Table", "aggBy": ":param aggregations: java.util.Collection\n:param groupByColumns: java.util.Collection\n:return: io.deephaven.engine.table.Table", "aj": "Looks up the columns in the rightTable that meet the match conditions in the columnsToMatch list. Matching is\n done exactly for the first n-1 columns and via a binary search for the last match pair. The columns of the\n original table are returned intact, together with the columns from rightTable defined in a comma separated list\n \"columnsToAdd\"\n\n:param rightTable: (io.deephaven.engine.table.Table) - The right side table on the join.\n:param columnsToMatch: (io.deephaven.engine.table.MatchPair[]) - A comma separated list of match conditions (\"leftColumn=rightColumn\" or\n \"columnFoundInBoth\")\n:param columnsToAdd: (io.deephaven.engine.table.MatchPair[]) - A comma separated list with the columns from the left side that need to be added to the right\n side as a result of the match.\n:param asOfMatchRule: io.deephaven.engine.table.Table.AsOfMatchRule\n:return: (io.deephaven.engine.table.Table) a new table joined according to the specification in columnsToMatch and columnsToAdd", - "copy": ":param copyAttributes: boolean\n:return: io.deephaven.engine.table.Table", + "copy": ":param shouldCopy: java.util.function.Predicate\n:return: io.deephaven.engine.table.Table", "countBy": ":param countColumnName: java.lang.String\n:param groupByColumns: io.deephaven.api.Selectable...\n:return: io.deephaven.engine.table.Table", "dropColumns": ":param columnNames: java.lang.String...\n:return: io.deephaven.engine.table.Table", "exactJoin": ":param table: io.deephaven.engine.table.Table\n:param columnsToMatch: io.deephaven.engine.table.MatchPair[]\n:param columnsToAdd: io.deephaven.engine.table.MatchPair[]\n:return: io.deephaven.engine.table.Table", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/PrevColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/PrevColumnSource.json index 94a0112bebf..7f366d9d9d7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/PrevColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/PrevColumnSource.json @@ -27,8 +27,10 @@ "getShort": ":param index: long\n:return: short", "getType": ":return: java.lang.Class", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", - "makeGetContext": "Allocate a new ChunkSource.GetContext for retrieving chunks from this GetContextMaker, typically\n a ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size required for any WritableChunk allocated as part of the result.\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.GetContext) A context for use with get operations" + "makeGetContext": "Allocate a new ChunkSource.GetContext for retrieving chunks from this GetContextMaker, typically\n a ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size required for any WritableChunk allocated as part of the result.\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.GetContext) A context for use with get operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.PrevColumnSource", "text": "Wrapper ColumnSource that maps current data accessors to previous data accessors (and disables previous data\n accessors).", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json index b3ef7ae6ec9..b31412a384f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/QueryTable.json @@ -5,7 +5,7 @@ "aggBy": ":param aggregations: java.util.Collection\n:param groupByColumns: java.util.Collection\n:return: io.deephaven.engine.table.Table", "aj": "Looks up the columns in the rightTable that meet the match conditions in the columnsToMatch list. Matching is\n done exactly for the first n-1 columns and via a binary search for the last match pair. The columns of the\n original table are returned intact, together with the columns from rightTable defined in a comma separated list\n \"columnsToAdd\"\n\n:param rightTable: (io.deephaven.engine.table.Table) - The right side table on the join.\n:param columnsToMatch: (io.deephaven.engine.table.MatchPair[]) - A comma separated list of match conditions (\"leftColumn=rightColumn\" or\n \"columnFoundInBoth\")\n:param columnsToAdd: (io.deephaven.engine.table.MatchPair[]) - A comma separated list with the columns from the left side that need to be added to the right\n side as a result of the match.\n:param asOfMatchRule: io.deephaven.engine.table.Table.AsOfMatchRule\n:return: (io.deephaven.engine.table.Table) a new table joined according to the specification in columnsToMatch and columnsToAdd", "apply": "Applies a function to this table.\n \n This is useful if you have a reference to a table or a proxy and want to run a series of operations against the\n table without each individual operation resulting in an RMI.\n\nNote: Java generics information - \n\n:param function: (java.util.function.Function) - the function to run, its single argument will be this table\n:return: (R) the return value of function", - "copy": "Copies this table, but with a new set of attributes.\n\n*Overload 1* \n :return: (io.deephaven.engine.table.Table) an identical table; but with a new set of attributes\n \n*Overload 2* \n :param copyAttributes: boolean\n :return: io.deephaven.engine.table.Table\n \n*Overload 3* \n :param definition: io.deephaven.engine.table.TableDefinition\n :param copyAttributes: boolean\n :return: io.deephaven.engine.table.Table", + "copy": "Copies this table, but with a new set of attributes.\n\n*Overload 1* \n :return: (io.deephaven.engine.table.Table) an identical table; but with a new set of attributes\n \n*Overload 2* \n :param shouldCopy: java.util.function.Predicate\n :return: io.deephaven.engine.table.Table\n \n*Overload 3* \n :param definition: io.deephaven.engine.table.TableDefinition\n :param shouldCopy: java.util.function.Predicate\n :return: io.deephaven.engine.table.Table", "countBy": ":param countColumnName: java.lang.String\n:param groupByColumns: io.deephaven.api.Selectable...\n:return: io.deephaven.engine.table.Table", "dateTimeColumnAsNanos": "Produce a new table with the same columns as this table, but with a new column presenting the specified DateTime\n column as a Long column (with each DateTime represented instead as the corresponding number of nanos since the\n epoch).\n \n NOTE: This is a really just an updateView(), and behaves accordingly for column ordering and (re)placement. This\n doesn't work on data that has been brought fully into memory (e.g. via select()). Use a view instead.\n\n:param dateTimeColumnName: (java.lang.String) - Name of date time column\n:param nanosColumnName: (java.lang.String) - Name of nanos column\n:return: (io.deephaven.engine.table.Table) The new table, constructed as explained above.", "dropColumns": ":param columnNames: java.lang.String...\n:return: io.deephaven.engine.table.Table", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/UnboxedDateTimeColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/UnboxedDateTimeColumnSource.json index e8421de781d..c477ff4187f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/UnboxedDateTimeColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/UnboxedDateTimeColumnSource.json @@ -5,7 +5,9 @@ "doReinterpret": "Supply allowed reinterpret results. The default implementation handles the most common case to avoid code\n duplication.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternate data type\n:return: (io.deephaven.engine.table.ColumnSource) The resulting ColumnSource", "getLong": ":param index: long\n:return: long", "getPrevLong": ":param index: long\n:return: long", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.UnboxedDateTimeColumnSource", "text": "Reinterpret result for many ColumnSource implementations that internally represent DateTime values as\n long values.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationHelper.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationHelper.json deleted file mode 100644 index b9d5482ef27..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/AggregationHelper.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.AggregationHelper", - "methods": { - "by": "*Overload 1* \n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param keyColumns: io.deephaven.engine.table.impl.select.SelectColumn...\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 2* \n :param aggregationControl: io.deephaven.engine.table.impl.by.AggregationControl\n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param keyColumns: io.deephaven.engine.table.impl.select.SelectColumn...\n :return: io.deephaven.engine.table.impl.QueryTable", - "partitionBy": "*Overload 1* \n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param dropKeyColumns: boolean\n :param keyColumnNames: java.lang.String...\n :return: io.deephaven.engine.table.impl.LocalTableMap\n \n*Overload 2* \n :param aggregationControl: io.deephaven.engine.table.impl.by.AggregationControl\n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param dropKeyColumns: boolean\n :param keyColumnNames: java.lang.String...\n :return: io.deephaven.engine.table.impl.LocalTableMap" - }, - "path": "io.deephaven.engine.table.impl.by.AggregationHelper", - "text": "Implementation for chunk-oriented aggregation operations, including Table.groupBy(java.util.Collection) and\n Table.partitionBy(boolean, java.lang.String...).", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.json deleted file mode 100644 index 5aef2a96f18..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FirstOrLastByAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.FirstOrLastByAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.FirstOrLastByAggregationFactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.json deleted file mode 100644 index 53c51a67d79..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FormulaAggregationFactory.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.FormulaAggregationFactory", - "methods": { - "applyToAllBy": "*Overload 1* \n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param formula: java.lang.String\n :param columnParamName: java.lang.String\n :param groupByColumnNames: java.lang.String...\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 2* \n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param formula: java.lang.String\n :param columnParamName: java.lang.String\n :param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 3* \n :param aggregationControl: io.deephaven.engine.table.impl.by.AggregationControl\n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param formula: java.lang.String\n :param columnParamName: java.lang.String\n :param groupByColumnNames: java.lang.String...\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 4* \n :param aggregationControl: io.deephaven.engine.table.impl.by.AggregationControl\n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param formula: java.lang.String\n :param columnParamName: java.lang.String\n :param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n :return: io.deephaven.engine.table.impl.QueryTable", - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param inputTable: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumnNames: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.FormulaAggregationFactory", - "text": "An AggregationContextFactory used in the implementation of Table.applyToAllBy(java.lang.String, java.lang.String, java.util.Collection).", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.json deleted file mode 100644 index 487704321aa..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/FreezeByAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.FreezeByAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.FreezeByAggregationFactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.json deleted file mode 100644 index 8f6b88fad5c..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/GroupByAggregationFactory.json +++ /dev/null @@ -1,12 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.GroupByAggregationFactory", - "methods": { - "by": "*Overload 1* \n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param groupByColumnNames: java.lang.String...\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 2* \n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 3* \n :param aggregationControl: io.deephaven.engine.table.impl.by.AggregationControl\n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param groupByColumnNames: java.lang.String...\n :return: io.deephaven.engine.table.impl.QueryTable\n \n*Overload 4* \n :param aggregationControl: io.deephaven.engine.table.impl.by.AggregationControl\n :param inputTable: io.deephaven.engine.table.impl.QueryTable\n :param groupByColumns: io.deephaven.engine.table.impl.select.SelectColumn[]\n :return: io.deephaven.engine.table.impl.QueryTable", - "getInstance": ":return: io.deephaven.engine.table.impl.by.AggregationContextFactory", - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param inputTable: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumnNames: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.GroupByAggregationFactory", - "text": "An AggregationContextFactory used in the implementation of Table.groupBy(java.util.Collection).", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.json deleted file mode 100644 index 1f18da672e2..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/IncrementalChunkedByAggregationStateManager.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.IncrementalChunkedByAggregationStateManager", - "methods": { - "doRehash": ":param bc: io.deephaven.engine.table.impl.by.IncrementalChunkedByAggregationStateManager.BuildContext\n:param moveCallback: io.deephaven.engine.table.impl.by.IncrementalChunkedByAggregationStateManager.StateMoveCallback", - "rehashRequired": ":return: boolean" - }, - "path": "io.deephaven.engine.table.impl.by.IncrementalChunkedByAggregationStateManager", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.json deleted file mode 100644 index 0b82b95dcc4..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/KeyOnlyAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.KeyOnlyAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext" - }, - "path": "io.deephaven.engine.table.impl.by.KeyOnlyAggregationFactory", - "text": "Simplest factory that has no operators or results, used to implement selectDistinct.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.json deleted file mode 100644 index 58ed09afc68..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/StaticChunkedByAggregationStateManager.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.StaticChunkedByAggregationStateManager", - "methods": { - "doRehash": ":param bc: io.deephaven.engine.table.impl.by.StaticChunkedByAggregationStateManager.BuildContext\n:param sourceChunkInsertedHashSlots: io.deephaven.chunk.WritableIntChunk\n:param insertedHashSlots: io.deephaven.engine.table.impl.sources.IntegerArraySource", - "rehashRequired": ":return: boolean" - }, - "path": "io.deephaven.engine.table.impl.by.StaticChunkedByAggregationStateManager", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.json deleted file mode 100644 index a3d7fe5aeb5..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/WeightedAverageSumAggregationFactory.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.engine.table.impl.by.WeightedAverageSumAggregationFactory", - "methods": { - "makeAggregationContext": "Make an AggregationContext for this aggregation.\n\n:param table: (io.deephaven.engine.table.Table) - The source Table to aggregate\n:param groupByColumns: (java.lang.String...) - The key column names\n:return: (io.deephaven.engine.table.impl.by.AggregationContext) A new or safely reusable AggregationContext", - "toString": ":return: java.lang.String" - }, - "path": "io.deephaven.engine.table.impl.by.WeightedAverageSumAggregationFactory", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/lang/QueryLanguageFunctionUtils.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/lang/QueryLanguageFunctionUtils.json index da4fafbe04f..26497bdcd9c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/lang/QueryLanguageFunctionUtils.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/lang/QueryLanguageFunctionUtils.json @@ -10,36 +10,36 @@ "bytePyCast": ":param a: java.lang.Object\n:return: byte", "charCast": "*Overload 1* \n :param a: int\n :return: char\n \n*Overload 2* \n :param a: double\n :return: char\n \n*Overload 3* \n :param a: long\n :return: char\n \n*Overload 4* \n :param a: float\n :return: char\n \n*Overload 5* \n :param a: byte\n :return: char\n \n*Overload 6* \n :param a: short\n :return: char\n \n*Overload 7* \n :param a: java.lang.Object\n :return: char", "charPyCast": ":param a: java.lang.Object\n:return: char", - "compareTo": "*Overload 1* \n :param obj1: java.lang.Comparable\n :param obj2: java.lang.Comparable\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 3* \n :param a: int\n :param b: double\n :return: int\n \n*Overload 4* \n :param a: int\n :param b: long\n :return: int\n \n*Overload 5* \n :param a: int\n :param b: float\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 8* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 9* \n :param a: double\n :param b: int\n :return: int\n \n*Overload 10* \n :param a: double\n :param b: double\n :return: int\n \n*Overload 11* \n :param a: double\n :param b: long\n :return: int\n \n*Overload 12* \n :param a: double\n :param b: float\n :return: int\n \n*Overload 13* \n :param a: double\n :param b: char\n :return: int\n \n*Overload 14* \n :param a: double\n :param b: byte\n :return: int\n \n*Overload 15* \n :param a: double\n :param b: short\n :return: int\n \n*Overload 16* \n :param a: long\n :param b: int\n :return: int\n \n*Overload 17* \n :param a: long\n :param b: double\n :return: int\n \n*Overload 18* \n :param a: long\n :param b: long\n :return: int\n \n*Overload 19* \n :param a: long\n :param b: float\n :return: int\n \n*Overload 20* \n :param a: long\n :param b: char\n :return: int\n \n*Overload 21* \n :param a: long\n :param b: byte\n :return: int\n \n*Overload 22* \n :param a: long\n :param b: short\n :return: int\n \n*Overload 23* \n :param a: float\n :param b: int\n :return: int\n \n*Overload 24* \n :param a: float\n :param b: double\n :return: int\n \n*Overload 25* \n :param a: float\n :param b: long\n :return: int\n \n*Overload 26* \n :param a: float\n :param b: float\n :return: int\n \n*Overload 27* \n :param a: float\n :param b: char\n :return: int\n \n*Overload 28* \n :param a: float\n :param b: byte\n :return: int\n \n*Overload 29* \n :param a: float\n :param b: short\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 31* \n :param a: char\n :param b: double\n :return: int\n \n*Overload 32* \n :param a: char\n :param b: long\n :return: int\n \n*Overload 33* \n :param a: char\n :param b: float\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 36* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 38* \n :param a: byte\n :param b: double\n :return: int\n \n*Overload 39* \n :param a: byte\n :param b: long\n :return: int\n \n*Overload 40* \n :param a: byte\n :param b: float\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 43* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 45* \n :param a: short\n :param b: double\n :return: int\n \n*Overload 46* \n :param a: short\n :param b: long\n :return: int\n \n*Overload 47* \n :param a: short\n :param b: float\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 50* \n :param a: short\n :param b: short\n :return: int", - "divide": "*Overload 1* \n :param a: int\n :param b: int\n :return: double\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: double\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: double\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: double\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: double\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: double\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: double\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: double\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: double\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: double\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: double\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: double\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: double\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: double\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: double\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: double\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: double\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: double\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: double\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: double\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: double\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: double\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: double\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: double\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: double\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: double\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: double\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: double\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: double\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: double", + "compareTo": "*Overload 1* \n :param obj1: java.lang.Comparable\n :param obj2: java.lang.Comparable\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 3* \n :param a: int\n :param b: double\n :return: int\n \n*Overload 4* \n :param a: int\n :param b: long\n :return: int\n \n*Overload 5* \n :param a: int\n :param b: float\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 8* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 9* \n :param a: double\n :param b: int\n :return: int\n \n*Overload 10* \n :param a: double\n :param b: double\n :return: int\n \n*Overload 11* \n :param a: double\n :param b: long\n :return: int\n \n*Overload 12* \n :param a: double\n :param b: float\n :return: int\n \n*Overload 13* \n :param a: double\n :param b: char\n :return: int\n \n*Overload 14* \n :param a: double\n :param b: byte\n :return: int\n \n*Overload 15* \n :param a: double\n :param b: short\n :return: int\n \n*Overload 16* \n :param a: long\n :param b: int\n :return: int\n \n*Overload 17* \n :param a: long\n :param b: double\n :return: int\n \n*Overload 18* \n :param a: long\n :param b: long\n :return: int\n \n*Overload 19* \n :param a: long\n :param b: float\n :return: int\n \n*Overload 20* \n :param a: long\n :param b: char\n :return: int\n \n*Overload 21* \n :param a: long\n :param b: byte\n :return: int\n \n*Overload 22* \n :param a: long\n :param b: short\n :return: int\n \n*Overload 23* \n :param a: float\n :param b: int\n :return: int\n \n*Overload 24* \n :param a: float\n :param b: double\n :return: int\n \n*Overload 25* \n :param a: float\n :param b: long\n :return: int\n \n*Overload 26* \n :param a: float\n :param b: float\n :return: int\n \n*Overload 27* \n :param a: float\n :param b: char\n :return: int\n \n*Overload 28* \n :param a: float\n :param b: byte\n :return: int\n \n*Overload 29* \n :param a: float\n :param b: short\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 31* \n :param a: char\n :param b: double\n :return: int\n \n*Overload 32* \n :param a: char\n :param b: long\n :return: int\n \n*Overload 33* \n :param a: char\n :param b: float\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 36* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 38* \n :param a: byte\n :param b: double\n :return: int\n \n*Overload 39* \n :param a: byte\n :param b: long\n :return: int\n \n*Overload 40* \n :param a: byte\n :param b: float\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 43* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 45* \n :param a: short\n :param b: double\n :return: int\n \n*Overload 46* \n :param a: short\n :param b: long\n :return: int\n \n*Overload 47* \n :param a: short\n :param b: float\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 50* \n :param a: short\n :param b: short\n :return: int\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: int\n \n*Overload 52* \n :param a: java.math.BigDecimal\n :param b: long\n :return: int\n \n*Overload 53* \n :param a: long\n :param b: java.math.BigDecimal\n :return: int\n \n*Overload 54* \n :param a: java.math.BigDecimal\n :param b: int\n :return: int\n \n*Overload 55* \n :param a: int\n :param b: java.math.BigDecimal\n :return: int\n \n*Overload 56* \n :param a: java.math.BigDecimal\n :param b: short\n :return: int\n \n*Overload 57* \n :param a: short\n :param b: java.math.BigDecimal\n :return: int\n \n*Overload 58* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: int\n \n*Overload 59* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: int\n \n*Overload 60* \n :param a: java.math.BigDecimal\n :param b: double\n :return: int\n \n*Overload 61* \n :param a: double\n :param b: java.math.BigDecimal\n :return: int\n \n*Overload 62* \n :param a: java.math.BigDecimal\n :param b: float\n :return: int\n \n*Overload 63* \n :param a: float\n :param b: java.math.BigDecimal\n :return: int", + "divide": "*Overload 1* \n :param a: int\n :param b: int\n :return: double\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: double\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: double\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: double\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: double\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: double\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: double\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: double\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: double\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: double\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: double\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: double\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: double\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: double\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: double\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: double\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: double\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: double\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: double\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: double\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: double\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: double\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: double\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: double\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: double\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: double\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: double\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: double\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: double\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: double\n \n*Overload 50* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: long\n :return: java.math.BigDecimal\n \n*Overload 52* \n :param a: long\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 53* \n :param a: java.math.BigDecimal\n :param b: int\n :return: java.math.BigDecimal\n \n*Overload 54* \n :param a: int\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 55* \n :param a: java.math.BigDecimal\n :param b: short\n :return: java.math.BigDecimal\n \n*Overload 56* \n :param a: short\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 57* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: java.math.BigDecimal\n \n*Overload 58* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 59* \n :param a: java.math.BigDecimal\n :param b: double\n :return: java.math.BigDecimal\n \n*Overload 60* \n :param a: double\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 61* \n :param a: java.math.BigDecimal\n :param b: float\n :return: java.math.BigDecimal\n \n*Overload 62* \n :param a: float\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal", "divideArray": "There are 147 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: double[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: double[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: double[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: double[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: double[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: double[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: double[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: double[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: double[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: float[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: float[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: float[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: double[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: double[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: double[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: double[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: double[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: double[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: double[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: double[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: double[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: double[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: double[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: double[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: double[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: double[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: double[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: double[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: double[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: double[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: double[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: double[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: double[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: double[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: double[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: double[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: double[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: double[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: double[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: double[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: double[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: double[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: double[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: double[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: double[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: double[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: double[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: double[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: double[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: double[]", "doBooleanPyCast": ":param a: java.lang.Object\n:return: java.lang.Boolean", "doStringPyCast": ":param a: java.lang.Object\n:return: java.lang.String", "doubleCast": "*Overload 1* \n :param a: int\n :return: double\n \n*Overload 2* \n :param a: long\n :return: double\n \n*Overload 3* \n :param a: float\n :return: double\n \n*Overload 4* \n :param a: char\n :return: double\n \n*Overload 5* \n :param a: byte\n :return: double\n \n*Overload 6* \n :param a: short\n :return: double\n \n*Overload 7* \n :param a: java.lang.Object\n :return: double", "doublePyCast": ":param a: java.lang.Object\n:return: double", - "eq": "*Overload 1* \n :param obj1: java.lang.Object\n :param obj2: java.lang.Object\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 8* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 15* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 22* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 29* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 36* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 43* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 50* \n :param a: short\n :param b: short\n :return: boolean", + "eq": "*Overload 1* \n :param obj1: java.lang.Object\n :param obj2: java.lang.Object\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 8* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 15* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 22* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 29* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 36* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 43* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 50* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 52* \n :param a: java.math.BigDecimal\n :param b: long\n :return: boolean\n \n*Overload 53* \n :param a: long\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 54* \n :param a: java.math.BigDecimal\n :param b: int\n :return: boolean\n \n*Overload 55* \n :param a: int\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 56* \n :param a: java.math.BigDecimal\n :param b: short\n :return: boolean\n \n*Overload 57* \n :param a: short\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 58* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: boolean\n \n*Overload 59* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 60* \n :param a: java.math.BigDecimal\n :param b: double\n :return: boolean\n \n*Overload 61* \n :param a: double\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 62* \n :param a: java.math.BigDecimal\n :param b: float\n :return: boolean\n \n*Overload 63* \n :param a: float\n :param b: java.math.BigDecimal\n :return: boolean", "eqArray": "There are 155 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: boolean[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: boolean[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: boolean[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: boolean[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: boolean[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: boolean[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: boolean[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: boolean[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: boolean[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: boolean[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: boolean[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: boolean[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: boolean[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: boolean[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: boolean[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: boolean[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: boolean[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: boolean[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: boolean[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: boolean[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: boolean[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: boolean[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: boolean[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: boolean[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: boolean[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: boolean[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: boolean[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: boolean[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: boolean[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: boolean[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: boolean[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: boolean[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: boolean[]", "floatCast": "*Overload 1* \n :param a: int\n :return: float\n \n*Overload 2* \n :param a: double\n :return: float\n \n*Overload 3* \n :param a: long\n :return: float\n \n*Overload 4* \n :param a: char\n :return: float\n \n*Overload 5* \n :param a: byte\n :return: float\n \n*Overload 6* \n :param a: short\n :return: float\n \n*Overload 7* \n :param a: java.lang.Object\n :return: float", "floatPyCast": ":param a: java.lang.Object\n:return: float", - "greater": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean", + "greater": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 52* \n :param a: java.math.BigDecimal\n :param b: long\n :return: boolean\n \n*Overload 53* \n :param a: long\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 54* \n :param a: java.math.BigDecimal\n :param b: int\n :return: boolean\n \n*Overload 55* \n :param a: int\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 56* \n :param a: java.math.BigDecimal\n :param b: short\n :return: boolean\n \n*Overload 57* \n :param a: short\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 58* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: boolean\n \n*Overload 59* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 60* \n :param a: java.math.BigDecimal\n :param b: double\n :return: boolean\n \n*Overload 61* \n :param a: double\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 62* \n :param a: java.math.BigDecimal\n :param b: float\n :return: boolean\n \n*Overload 63* \n :param a: float\n :param b: java.math.BigDecimal\n :return: boolean", "greaterArray": "There are 150 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: boolean[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: boolean[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: boolean[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: boolean[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: boolean[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: boolean[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: boolean[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: boolean[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: boolean[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: boolean[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: boolean[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: boolean[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: boolean[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: boolean[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: boolean[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: boolean[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: boolean[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: boolean[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: boolean[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: boolean[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: boolean[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: boolean[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: boolean[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: boolean[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: boolean[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: boolean[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: boolean[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: boolean[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: boolean[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: boolean[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: boolean[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: boolean[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: boolean[]", - "greaterEquals": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean", + "greaterEquals": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 52* \n :param a: java.math.BigDecimal\n :param b: long\n :return: boolean\n \n*Overload 53* \n :param a: long\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 54* \n :param a: java.math.BigDecimal\n :param b: int\n :return: boolean\n \n*Overload 55* \n :param a: int\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 56* \n :param a: java.math.BigDecimal\n :param b: short\n :return: boolean\n \n*Overload 57* \n :param a: short\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 58* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: boolean\n \n*Overload 59* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 60* \n :param a: java.math.BigDecimal\n :param b: double\n :return: boolean\n \n*Overload 61* \n :param a: double\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 62* \n :param a: java.math.BigDecimal\n :param b: float\n :return: boolean\n \n*Overload 63* \n :param a: float\n :param b: java.math.BigDecimal\n :return: boolean", "greaterEqualsArray": "There are 150 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: boolean[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: boolean[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: boolean[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: boolean[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: boolean[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: boolean[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: boolean[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: boolean[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: boolean[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: boolean[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: boolean[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: boolean[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: boolean[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: boolean[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: boolean[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: boolean[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: boolean[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: boolean[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: boolean[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: boolean[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: boolean[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: boolean[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: boolean[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: boolean[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: boolean[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: boolean[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: boolean[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: boolean[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: boolean[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: boolean[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: boolean[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: boolean[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: boolean[]", "intCast": "*Overload 1* \n :param a: double\n :return: int\n \n*Overload 2* \n :param a: long\n :return: int\n \n*Overload 3* \n :param a: float\n :return: int\n \n*Overload 4* \n :param a: char\n :return: int\n \n*Overload 5* \n :param a: byte\n :return: int\n \n*Overload 6* \n :param a: short\n :return: int\n \n*Overload 7* \n :param a: java.lang.Object\n :return: int", "intPyCast": ":param a: java.lang.Object\n:return: int", - "less": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean", + "less": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 52* \n :param a: java.math.BigDecimal\n :param b: long\n :return: boolean\n \n*Overload 53* \n :param a: long\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 54* \n :param a: java.math.BigDecimal\n :param b: int\n :return: boolean\n \n*Overload 55* \n :param a: int\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 56* \n :param a: java.math.BigDecimal\n :param b: short\n :return: boolean\n \n*Overload 57* \n :param a: short\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 58* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: boolean\n \n*Overload 59* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 60* \n :param a: java.math.BigDecimal\n :param b: double\n :return: boolean\n \n*Overload 61* \n :param a: double\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 62* \n :param a: java.math.BigDecimal\n :param b: float\n :return: boolean\n \n*Overload 63* \n :param a: float\n :param b: java.math.BigDecimal\n :return: boolean", "lessArray": "There are 150 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: boolean[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: boolean[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: boolean[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: boolean[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: boolean[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: boolean[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: boolean[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: boolean[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: boolean[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: boolean[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: boolean[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: boolean[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: boolean[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: boolean[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: boolean[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: boolean[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: boolean[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: boolean[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: boolean[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: boolean[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: boolean[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: boolean[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: boolean[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: boolean[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: boolean[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: boolean[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: boolean[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: boolean[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: boolean[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: boolean[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: boolean[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: boolean[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: boolean[]", - "lessEquals": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean", + "lessEquals": "*Overload 1* \n :param a: int\n :param b: int\n :return: boolean\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: boolean\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: boolean\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: boolean\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: boolean\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: boolean\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: boolean\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: boolean\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: boolean\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: boolean\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: boolean\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: boolean\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: boolean\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: boolean\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: boolean\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: boolean\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: boolean\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: boolean\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: boolean\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: boolean\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: boolean\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: boolean\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: boolean\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: boolean\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: boolean\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: boolean\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: boolean\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: boolean\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: boolean\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: boolean\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: boolean\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: boolean\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: boolean\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: boolean\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: boolean\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: boolean\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: boolean\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: boolean\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: boolean\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: boolean\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: boolean\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: boolean\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: boolean\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: boolean\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: boolean\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: boolean\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: boolean\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: boolean\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: boolean\n \n*Overload 50* \n :param a: java.lang.Comparable\n :param b: java.lang.Comparable\n :return: boolean\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 52* \n :param a: java.math.BigDecimal\n :param b: long\n :return: boolean\n \n*Overload 53* \n :param a: long\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 54* \n :param a: java.math.BigDecimal\n :param b: int\n :return: boolean\n \n*Overload 55* \n :param a: int\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 56* \n :param a: java.math.BigDecimal\n :param b: short\n :return: boolean\n \n*Overload 57* \n :param a: short\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 58* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: boolean\n \n*Overload 59* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 60* \n :param a: java.math.BigDecimal\n :param b: double\n :return: boolean\n \n*Overload 61* \n :param a: double\n :param b: java.math.BigDecimal\n :return: boolean\n \n*Overload 62* \n :param a: java.math.BigDecimal\n :param b: float\n :return: boolean\n \n*Overload 63* \n :param a: float\n :param b: java.math.BigDecimal\n :return: boolean", "lessEqualsArray": "There are 150 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: boolean[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: boolean[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: boolean[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: boolean[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: boolean[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: boolean[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: boolean[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: boolean[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: boolean[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: boolean[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: boolean[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: boolean[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: boolean[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: boolean[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: boolean[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: boolean[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: boolean[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: boolean[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: boolean[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: boolean[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: boolean[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: boolean[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: boolean[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: boolean[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: boolean[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: boolean[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: boolean[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: boolean[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: boolean[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: boolean[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: boolean[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: boolean[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: boolean[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: boolean[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: boolean[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: boolean[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: boolean[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: boolean[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: boolean[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: boolean[]", "longCast": "*Overload 1* \n :param a: int\n :return: long\n \n*Overload 2* \n :param a: double\n :return: long\n \n*Overload 3* \n :param a: float\n :return: long\n \n*Overload 4* \n :param a: char\n :return: long\n \n*Overload 5* \n :param a: byte\n :return: long\n \n*Overload 6* \n :param a: short\n :return: long\n \n*Overload 7* \n :param a: java.lang.Object\n :return: long", "longPyCast": ":param a: java.lang.Object\n:return: long", - "minus": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int", + "minus": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int\n \n*Overload 50* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: long\n :return: java.math.BigDecimal\n \n*Overload 52* \n :param a: long\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 53* \n :param a: java.math.BigDecimal\n :param b: int\n :return: java.math.BigDecimal\n \n*Overload 54* \n :param a: int\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 55* \n :param a: java.math.BigDecimal\n :param b: short\n :return: java.math.BigDecimal\n \n*Overload 56* \n :param a: short\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 57* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: java.math.BigDecimal\n \n*Overload 58* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 59* \n :param a: java.math.BigDecimal\n :param b: double\n :return: java.math.BigDecimal\n \n*Overload 60* \n :param a: double\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 61* \n :param a: java.math.BigDecimal\n :param b: float\n :return: java.math.BigDecimal\n \n*Overload 62* \n :param a: float\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal", "minusArray": "There are 147 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: int[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: int[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: int[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: double[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: double[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: double[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: long[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: long[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: long[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: float[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: float[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: float[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: int[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: int[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: int[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: int[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: int[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: int[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: int[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: int[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: int[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: double[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: double[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: double[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: double[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: double[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: double[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: double[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: double[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: double[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: double[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: double[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: double[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: double[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: double[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: double[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: double[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: double[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: double[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: double[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: double[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: double[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: long[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: long[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: long[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: double[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: double[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: double[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: long[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: long[]", - "multiply": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int", + "multiply": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int\n \n*Overload 50* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: long\n :return: java.math.BigDecimal\n \n*Overload 52* \n :param a: long\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 53* \n :param a: java.math.BigDecimal\n :param b: int\n :return: java.math.BigDecimal\n \n*Overload 54* \n :param a: int\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 55* \n :param a: java.math.BigDecimal\n :param b: short\n :return: java.math.BigDecimal\n \n*Overload 56* \n :param a: short\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 57* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: java.math.BigDecimal\n \n*Overload 58* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 59* \n :param a: java.math.BigDecimal\n :param b: double\n :return: java.math.BigDecimal\n \n*Overload 60* \n :param a: double\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 61* \n :param a: java.math.BigDecimal\n :param b: float\n :return: java.math.BigDecimal\n \n*Overload 62* \n :param a: float\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal", "multiplyArray": "There are 147 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: int[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: int[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: int[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: double[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: double[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: double[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: long[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: long[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: long[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: float[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: float[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: float[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: int[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: int[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: int[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: int[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: int[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: int[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: int[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: int[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: int[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: double[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: double[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: double[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: double[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: double[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: double[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: double[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: double[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: double[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: double[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: double[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: double[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: double[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: double[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: double[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: double[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: double[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: double[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: double[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: double[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: double[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: long[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: long[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: long[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: double[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: double[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: double[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: long[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: long[]", "negate": "*Overload 1* \n :param a: int\n :return: int\n \n*Overload 2* \n :param a: double\n :return: double\n \n*Overload 3* \n :param a: long\n :return: long\n \n*Overload 4* \n :param a: float\n :return: float\n \n*Overload 5* \n :param a: char\n :return: int\n \n*Overload 6* \n :param a: byte\n :return: int\n \n*Overload 7* \n :param a: short\n :return: int", "not": ":param a: java.lang.Boolean\n:return: java.lang.Boolean", - "plus": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int", + "plus": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int\n \n*Overload 50* \n :param a: java.math.BigDecimal\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 51* \n :param a: java.math.BigDecimal\n :param b: long\n :return: java.math.BigDecimal\n \n*Overload 52* \n :param a: long\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 53* \n :param a: java.math.BigDecimal\n :param b: int\n :return: java.math.BigDecimal\n \n*Overload 54* \n :param a: int\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 55* \n :param a: java.math.BigDecimal\n :param b: short\n :return: java.math.BigDecimal\n \n*Overload 56* \n :param a: short\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 57* \n :param a: java.math.BigDecimal\n :param b: byte\n :return: java.math.BigDecimal\n \n*Overload 58* \n :param a: byte\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 59* \n :param a: java.math.BigDecimal\n :param b: double\n :return: java.math.BigDecimal\n \n*Overload 60* \n :param a: double\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal\n \n*Overload 61* \n :param a: java.math.BigDecimal\n :param b: float\n :return: java.math.BigDecimal\n \n*Overload 62* \n :param a: float\n :param b: java.math.BigDecimal\n :return: java.math.BigDecimal", "plusArray": "There are 147 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: int[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: int[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: int[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: double[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: double[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: double[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: long[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: long[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: long[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: float[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: float[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: float[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: int[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: int[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: int[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: int[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: int[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: int[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: int[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: int[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: int[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: double[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: double[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: double[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: double[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: double[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: double[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: double[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: double[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: double[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: double[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: double[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: double[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: double[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: double[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: double[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: double[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: double[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: double[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: double[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: double[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: double[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: long[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: long[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: long[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: double[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: double[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: double[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: long[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: long[]", "remainder": "*Overload 1* \n :param a: int\n :param b: int\n :return: int\n \n*Overload 2* \n :param a: int\n :param b: double\n :return: double\n \n*Overload 3* \n :param a: int\n :param b: long\n :return: long\n \n*Overload 4* \n :param a: int\n :param b: float\n :return: float\n \n*Overload 5* \n :param a: int\n :param b: char\n :return: int\n \n*Overload 6* \n :param a: int\n :param b: byte\n :return: int\n \n*Overload 7* \n :param a: int\n :param b: short\n :return: int\n \n*Overload 8* \n :param a: double\n :param b: int\n :return: double\n \n*Overload 9* \n :param a: double\n :param b: double\n :return: double\n \n*Overload 10* \n :param a: double\n :param b: long\n :return: double\n \n*Overload 11* \n :param a: double\n :param b: float\n :return: double\n \n*Overload 12* \n :param a: double\n :param b: char\n :return: double\n \n*Overload 13* \n :param a: double\n :param b: byte\n :return: double\n \n*Overload 14* \n :param a: double\n :param b: short\n :return: double\n \n*Overload 15* \n :param a: long\n :param b: int\n :return: long\n \n*Overload 16* \n :param a: long\n :param b: double\n :return: double\n \n*Overload 17* \n :param a: long\n :param b: long\n :return: long\n \n*Overload 18* \n :param a: long\n :param b: float\n :return: float\n \n*Overload 19* \n :param a: long\n :param b: char\n :return: long\n \n*Overload 20* \n :param a: long\n :param b: byte\n :return: long\n \n*Overload 21* \n :param a: long\n :param b: short\n :return: long\n \n*Overload 22* \n :param a: float\n :param b: int\n :return: float\n \n*Overload 23* \n :param a: float\n :param b: double\n :return: double\n \n*Overload 24* \n :param a: float\n :param b: long\n :return: float\n \n*Overload 25* \n :param a: float\n :param b: float\n :return: float\n \n*Overload 26* \n :param a: float\n :param b: char\n :return: float\n \n*Overload 27* \n :param a: float\n :param b: byte\n :return: float\n \n*Overload 28* \n :param a: float\n :param b: short\n :return: float\n \n*Overload 29* \n :param a: char\n :param b: int\n :return: int\n \n*Overload 30* \n :param a: char\n :param b: double\n :return: double\n \n*Overload 31* \n :param a: char\n :param b: long\n :return: long\n \n*Overload 32* \n :param a: char\n :param b: float\n :return: float\n \n*Overload 33* \n :param a: char\n :param b: char\n :return: int\n \n*Overload 34* \n :param a: char\n :param b: byte\n :return: int\n \n*Overload 35* \n :param a: char\n :param b: short\n :return: int\n \n*Overload 36* \n :param a: byte\n :param b: int\n :return: int\n \n*Overload 37* \n :param a: byte\n :param b: double\n :return: double\n \n*Overload 38* \n :param a: byte\n :param b: long\n :return: long\n \n*Overload 39* \n :param a: byte\n :param b: float\n :return: float\n \n*Overload 40* \n :param a: byte\n :param b: char\n :return: int\n \n*Overload 41* \n :param a: byte\n :param b: byte\n :return: int\n \n*Overload 42* \n :param a: byte\n :param b: short\n :return: int\n \n*Overload 43* \n :param a: short\n :param b: int\n :return: int\n \n*Overload 44* \n :param a: short\n :param b: double\n :return: double\n \n*Overload 45* \n :param a: short\n :param b: long\n :return: long\n \n*Overload 46* \n :param a: short\n :param b: float\n :return: float\n \n*Overload 47* \n :param a: short\n :param b: char\n :return: int\n \n*Overload 48* \n :param a: short\n :param b: byte\n :return: int\n \n*Overload 49* \n :param a: short\n :param b: short\n :return: int", "remainderArray": "There are 147 overloads, restricting signature summary to first 50:\n*Overload 1* \n :param a: int[]\n :param b: int[]\n :return: int[]\n \n*Overload 2* \n :param a: int[]\n :param b: int\n :return: int[]\n \n*Overload 3* \n :param a: int\n :param b: int[]\n :return: int[]\n \n*Overload 4* \n :param a: int[]\n :param b: double[]\n :return: double[]\n \n*Overload 5* \n :param a: int[]\n :param b: double\n :return: double[]\n \n*Overload 6* \n :param a: int\n :param b: double[]\n :return: double[]\n \n*Overload 7* \n :param a: int[]\n :param b: long[]\n :return: long[]\n \n*Overload 8* \n :param a: int[]\n :param b: long\n :return: long[]\n \n*Overload 9* \n :param a: int\n :param b: long[]\n :return: long[]\n \n*Overload 10* \n :param a: int[]\n :param b: float[]\n :return: float[]\n \n*Overload 11* \n :param a: int[]\n :param b: float\n :return: float[]\n \n*Overload 12* \n :param a: int\n :param b: float[]\n :return: float[]\n \n*Overload 13* \n :param a: int[]\n :param b: char[]\n :return: int[]\n \n*Overload 14* \n :param a: int[]\n :param b: char\n :return: int[]\n \n*Overload 15* \n :param a: int\n :param b: char[]\n :return: int[]\n \n*Overload 16* \n :param a: int[]\n :param b: byte[]\n :return: int[]\n \n*Overload 17* \n :param a: int[]\n :param b: byte\n :return: int[]\n \n*Overload 18* \n :param a: int\n :param b: byte[]\n :return: int[]\n \n*Overload 19* \n :param a: int[]\n :param b: short[]\n :return: int[]\n \n*Overload 20* \n :param a: int[]\n :param b: short\n :return: int[]\n \n*Overload 21* \n :param a: int\n :param b: short[]\n :return: int[]\n \n*Overload 22* \n :param a: double[]\n :param b: int[]\n :return: double[]\n \n*Overload 23* \n :param a: double[]\n :param b: int\n :return: double[]\n \n*Overload 24* \n :param a: double\n :param b: int[]\n :return: double[]\n \n*Overload 25* \n :param a: double[]\n :param b: double[]\n :return: double[]\n \n*Overload 26* \n :param a: double[]\n :param b: double\n :return: double[]\n \n*Overload 27* \n :param a: double\n :param b: double[]\n :return: double[]\n \n*Overload 28* \n :param a: double[]\n :param b: long[]\n :return: double[]\n \n*Overload 29* \n :param a: double[]\n :param b: long\n :return: double[]\n \n*Overload 30* \n :param a: double\n :param b: long[]\n :return: double[]\n \n*Overload 31* \n :param a: double[]\n :param b: float[]\n :return: double[]\n \n*Overload 32* \n :param a: double[]\n :param b: float\n :return: double[]\n \n*Overload 33* \n :param a: double\n :param b: float[]\n :return: double[]\n \n*Overload 34* \n :param a: double[]\n :param b: char[]\n :return: double[]\n \n*Overload 35* \n :param a: double[]\n :param b: char\n :return: double[]\n \n*Overload 36* \n :param a: double\n :param b: char[]\n :return: double[]\n \n*Overload 37* \n :param a: double[]\n :param b: byte[]\n :return: double[]\n \n*Overload 38* \n :param a: double[]\n :param b: byte\n :return: double[]\n \n*Overload 39* \n :param a: double\n :param b: byte[]\n :return: double[]\n \n*Overload 40* \n :param a: double[]\n :param b: short[]\n :return: double[]\n \n*Overload 41* \n :param a: double[]\n :param b: short\n :return: double[]\n \n*Overload 42* \n :param a: double\n :param b: short[]\n :return: double[]\n \n*Overload 43* \n :param a: long[]\n :param b: int[]\n :return: long[]\n \n*Overload 44* \n :param a: long[]\n :param b: int\n :return: long[]\n \n*Overload 45* \n :param a: long\n :param b: int[]\n :return: long[]\n \n*Overload 46* \n :param a: long[]\n :param b: double[]\n :return: double[]\n \n*Overload 47* \n :param a: long[]\n :param b: double\n :return: double[]\n \n*Overload 48* \n :param a: long\n :param b: double[]\n :return: double[]\n \n*Overload 49* \n :param a: long[]\n :param b: long[]\n :return: long[]\n \n*Overload 50* \n :param a: long[]\n :param b: long\n :return: long[]", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/AbstractFormulaColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/AbstractFormulaColumn.json index 825d64ec289..e5f47d884d2 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/AbstractFormulaColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/AbstractFormulaColumn.json @@ -13,6 +13,7 @@ "isRetain": ":return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", + "preventsParallelization": ":return: boolean", "toString": ":return: java.lang.String" }, "path": "io.deephaven.engine.table.impl.select.AbstractFormulaColumn", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/DhFormulaColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/DhFormulaColumn.json index c6e9c0740ba..48647252486 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/DhFormulaColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/DhFormulaColumn.json @@ -4,7 +4,9 @@ "copy": "Create a copy of this SelectColumn.\n\n:return: (io.deephaven.engine.table.impl.select.SelectColumn) an independent copy of this SelectColumn.", "getFormulaColumnPython": ":return: io.deephaven.engine.table.impl.select.python.FormulaColumnPython", "getVectorType": ":param declaredType: java.lang.Class\n:return: java.lang.Class", - "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent" + "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", + "preventsParallelization": "Does this formula column use Python (which would cause us to hang the GIL if we evaluate it off thread?)\n\n:return: (boolean) true if this column has the potential to hang the gil" }, "path": "io.deephaven.engine.table.impl.select.DhFormulaColumn", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/FunctionalColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/FunctionalColumn.json index 0eecd2e5435..889ea46947f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/FunctionalColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/FunctionalColumn.json @@ -13,6 +13,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "toString": ":return: java.lang.String" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.json index 6456c4a49c3..d8072e26a7c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/MultiSourceFunctionalColumn.json @@ -13,6 +13,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "toString": ":return: java.lang.String" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/NullSelectColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/NullSelectColumn.json index dbf82604d7a..4993d4ab50e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/NullSelectColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/NullSelectColumn.json @@ -13,6 +13,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/ReinterpretedColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/ReinterpretedColumn.json index e9c8db01551..5c4415258c8 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/ReinterpretedColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/ReinterpretedColumn.json @@ -15,6 +15,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "toString": ":return: java.lang.String" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SelectColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SelectColumn.json index aa0cee5cd98..b5982a12cb9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SelectColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SelectColumn.json @@ -15,6 +15,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newColumn": "The new column name, to be added to the new table.\n\n:return: (io.deephaven.api.ColumnName) the new column name", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SourceColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SourceColumn.json index 1d90b61f286..ddb9dab3560 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SourceColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SourceColumn.json @@ -16,6 +16,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "toString": ":return: java.lang.String" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SwitchColumn.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SwitchColumn.json index 280e98696b1..4b575a4db71 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SwitchColumn.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/SwitchColumn.json @@ -14,6 +14,7 @@ "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnDefinitionMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", "initInputs": "**Incompatible overloads text - text from the first overload:**\n\nInitialize the SelectColumn using the input table and return a list of underlying columns that this SelectColumn\n is dependent upon.\n\n*Overload 1* \n :param table: (io.deephaven.engine.table.Table) - the table to initialize internals from\n :return: (java.util.List) a list containing all columns from 'table' that the result depends on\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - the base row set\n :param columnsOfInterest: (java.util.Map>) - the input columns\n :return: (java.util.List) a list of columns on which the result of this is dependent", "isRetain": ":return: boolean", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", "newDestInstance": "Create a new WritableColumnSource.\n\n The returned column source must be capable of handling updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "newFlatDestInstance": "Create a new immutable WritableColumnSource.\n\n The returned column source should be flat, and need not handle updates.\n\n:param size: (long) - A hint as to the number of rows that will be used\n:return: (io.deephaven.engine.table.WritableColumnSource) a new WritableColumnSource", "toString": ":return: java.lang.String" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.json index bac2e88dc4a..057df6f9850 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/BaseLayer.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.engine.table.impl.select.analyzers.BaseLayer", "methods": { + "allowCrossColumnParallelization": "Can all of our columns permit parallel updates?\n\n:return: boolean", "append": ":param logOutput: io.deephaven.base.log.LogOutput\n:return: io.deephaven.base.log.LogOutput", "applyUpdate": "Apply this update to this SelectAndViewAnalyzer.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - the upstream update\n:param toClear: (io.deephaven.engine.rowset.RowSet) - rows that used to exist and no longer exist\n:param helper: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateHelper) - convenience class that memoizes reusable calculations for this update\n:param jobScheduler: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler\n:param onCompletion: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.SelectLayerCompletionHandler) - Called when an inner column is complete. The outer layer should pass the onCompletion\n on to other layers and if it and all of its dependencies have been satisfied schedule execution of that\n column update.", "getInner": ":return: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.json index 59fcefa1aae..db79f779d2b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/PreserveColumnLayer.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.engine.table.impl.select.analyzers.PreserveColumnLayer", "methods": { + "allowCrossColumnParallelization": "Can all of our columns permit parallel updates?\n\n:return: boolean", "append": ":param logOutput: io.deephaven.base.log.LogOutput\n:return: io.deephaven.base.log.LogOutput", "applyUpdate": "Apply this update to this SelectAndViewAnalyzer.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - the upstream update\n:param toClear: (io.deephaven.engine.rowset.RowSet) - rows that used to exist and no longer exist\n:param helper: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateHelper) - convenience class that memoizes reusable calculations for this update\n:param jobScheduler: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler\n:param onCompletion: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.SelectLayerCompletionHandler) - Called when an inner column is complete. The outer layer should pass the onCompletion\n on to other layers and if it and all of its dependencies have been satisfied schedule execution of that\n column update." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.json index 0bc58748ee1..0ee576f812f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/RedirectionLayer.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.engine.table.impl.select.analyzers.RedirectionLayer", "methods": { + "allowCrossColumnParallelization": "Can all of our columns permit parallel updates?\n\n:return: boolean", "append": ":param logOutput: io.deephaven.base.log.LogOutput\n:return: io.deephaven.base.log.LogOutput", "applyUpdate": "Apply this update to this SelectAndViewAnalyzer.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - the upstream update\n:param toClear: (io.deephaven.engine.rowset.RowSet) - rows that used to exist and no longer exist\n:param helper: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateHelper) - convenience class that memoizes reusable calculations for this update\n:param jobScheduler: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler\n:param onCompletion: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.SelectLayerCompletionHandler) - Called when an inner column is complete. The outer layer should pass the onCompletion\n on to other layers and if it and all of its dependencies have been satisfied schedule execution of that\n column update.", "calcDependsOnRecurse": ":return: java.util.Map>", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.json index 112fa405aa3..10ae2a6b221 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer", "methods": { + "allowCrossColumnParallelization": "Can all of our columns permit parallel updates?\n\n:return: boolean", "applyUpdate": "Apply this update to this SelectAndViewAnalyzer.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - the upstream update\n:param toClear: (io.deephaven.engine.rowset.RowSet) - rows that used to exist and no longer exist\n:param helper: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateHelper) - convenience class that memoizes reusable calculations for this update\n:param jobScheduler: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler\n:param onCompletion: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.SelectLayerCompletionHandler) - Called when an inner column is complete. The outer layer should pass the onCompletion\n on to other layers and if it and all of its dependencies have been satisfied schedule execution of that\n column update.", "calcEffects": "Our job here is to calculate the effects: a map from incoming column to a list of columns that it effects. We do\n this in two stages. In the first stage we create a map from column to (set of dependent columns). In the second\n stage we reverse that map.\n\n:return: java.util.Map", "create": "*Overload 1* \n :param mode: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.Mode\n :param columnSources: java.util.Map>\n :param rowSet: io.deephaven.engine.rowset.TrackingRowSet\n :param parentMcs: io.deephaven.engine.table.ModifiedColumnSet\n :param publishTheseSources: boolean\n :param selectColumns: io.deephaven.engine.table.impl.select.SelectColumn...\n :return: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer\n \n*Overload 2* \n :param mode: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.Mode\n :param columnSources: java.util.Map>\n :param rowSet: io.deephaven.engine.rowset.TrackingRowSet\n :param parentMcs: io.deephaven.engine.table.ModifiedColumnSet\n :param publishTheseSources: boolean\n :param allowInternalFlatten: boolean\n :param selectColumns: io.deephaven.engine.table.impl.select.SelectColumn...\n :return: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/ImmediateJobScheduler.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/ImmediateJobScheduler.json index a6b3b30a8f7..285a68490c5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/ImmediateJobScheduler.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/ImmediateJobScheduler.json @@ -2,7 +2,8 @@ "className": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer$ImmediateJobScheduler", "methods": { "getAccumulatedPerformance": "The performance statistics of all runnables that have been completed off-thread, or null if it was executed\n in the current thread.\n\n:return: io.deephaven.engine.table.impl.perf.BasePerformanceEntry", - "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable" + "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable", + "threadCount": "How many threads exist in the job scheduler? The job submitters can use this value to determine how many\n sub-jobs to split work into.\n\n:return: int" }, "path": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.ImmediateJobScheduler", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/JobScheduler.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/JobScheduler.json index 6ab9d4ed673..9587e35fb17 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/JobScheduler.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/JobScheduler.json @@ -2,7 +2,8 @@ "className": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer$JobScheduler", "methods": { "getAccumulatedPerformance": "The performance statistics of all runnables that have been completed off-thread, or null if it was executed\n in the current thread.\n\n:return: io.deephaven.engine.table.impl.perf.BasePerformanceEntry", - "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable" + "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable", + "threadCount": "How many threads exist in the job scheduler? The job submitters can use this value to determine how many\n sub-jobs to split work into.\n\n:return: int" }, "path": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler", "text": "An interface for submitting jobs to be executed and accumulating their performance of all the tasks performed off\n thread.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/TableMapTransformJobScheduler.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/OperationInitializationPoolJobScheduler.json similarity index 69% rename from Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/TableMapTransformJobScheduler.json rename to Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/OperationInitializationPoolJobScheduler.json index b0d02fd6e8d..999a310bd38 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/TableMapTransformJobScheduler.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/OperationInitializationPoolJobScheduler.json @@ -1,9 +1,10 @@ { - "className": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer$TableMapTransformJobScheduler", + "className": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer$OperationInitializationPoolJobScheduler", "methods": { "getAccumulatedPerformance": "The performance statistics of all runnables that have been completed off-thread, or null if it was executed\n in the current thread.\n\n:return: io.deephaven.engine.table.impl.perf.BasePerformanceEntry", - "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable" + "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable", + "threadCount": "How many threads exist in the job scheduler? The job submitters can use this value to determine how many\n sub-jobs to split work into.\n\n:return: int" }, - "path": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.TableMapTransformJobScheduler", + "path": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.OperationInitializationPoolJobScheduler", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/UpdateGraphProcessorJobScheduler.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/UpdateGraphProcessorJobScheduler.json index 94e30aa5853..27dfbc148df 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/UpdateGraphProcessorJobScheduler.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectAndViewAnalyzer/UpdateGraphProcessorJobScheduler.json @@ -2,7 +2,8 @@ "className": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer$UpdateGraphProcessorJobScheduler", "methods": { "getAccumulatedPerformance": "The performance statistics of all runnables that have been completed off-thread, or null if it was executed\n in the current thread.\n\n:return: io.deephaven.engine.table.impl.perf.BasePerformanceEntry", - "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable" + "submit": "Cause runnable to be executed.\n\n:param runnable: (java.lang.Runnable) - the runnable to execute\n:param description: (io.deephaven.base.log.LogOutputAppendable) - a description for logging\n:param onError: (java.util.function.Consumer) - a routine to call if an exception occurs while running runnable", + "threadCount": "How many threads exist in the job scheduler? The job submitters can use this value to determine how many\n sub-jobs to split work into.\n\n:return: int" }, "path": "io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateGraphProcessorJobScheduler", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.json index 7737a8d486e..5898003e0a1 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/SelectColumnLayer.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.engine.table.impl.select.analyzers.SelectColumnLayer", "methods": { + "allowCrossColumnParallelization": "Can all of our columns permit parallel updates?\n\n:return: boolean", "append": ":param logOutput: io.deephaven.base.log.LogOutput\n:return: io.deephaven.base.log.LogOutput", "applyUpdate": "Apply this update to this SelectAndViewAnalyzer.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - the upstream update\n:param toClear: (io.deephaven.engine.rowset.RowSet) - rows that used to exist and no longer exist\n:param helper: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateHelper) - convenience class that memoizes reusable calculations for this update\n:param jobScheduler: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler\n:param onCompletion: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.SelectLayerCompletionHandler) - Called when an inner column is complete. The outer layer should pass the onCompletion\n on to other layers and if it and all of its dependencies have been satisfied schedule execution of that\n column update.", "flattenedResult": "Was the result internally flattened? Only the STATIC_SELECT case flattens the result. If the result preserves any\n columns, then flattening is not permitted. Because all the other layers cannot internally flatten, the default\n implementation returns false.\n\n:return: boolean" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.json index d1637ae0cbb..c317caed744 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/analyzers/ViewColumnLayer.json @@ -1,6 +1,7 @@ { "className": "io.deephaven.engine.table.impl.select.analyzers.ViewColumnLayer", "methods": { + "allowCrossColumnParallelization": "Can all of our columns permit parallel updates?\n\n:return: boolean", "append": ":param logOutput: io.deephaven.base.log.LogOutput\n:return: io.deephaven.base.log.LogOutput", "applyUpdate": "Apply this update to this SelectAndViewAnalyzer.\n\n:param upstream: (io.deephaven.engine.table.TableUpdate) - the upstream update\n:param toClear: (io.deephaven.engine.rowset.RowSet) - rows that used to exist and no longer exist\n:param helper: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.UpdateHelper) - convenience class that memoizes reusable calculations for this update\n:param jobScheduler: io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.JobScheduler\n:param completionHandler: (io.deephaven.engine.table.impl.select.analyzers.SelectAndViewAnalyzer.SelectLayerCompletionHandler) - Called when an inner column is complete. The outer layer should pass the onCompletion\n on to other layers and if it and all of its dependencies have been satisfied schedule execution of that\n column update." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/python/FormulaColumnPython.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/python/FormulaColumnPython.json index c6bdc7ba65c..179bc57d816 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/python/FormulaColumnPython.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/select/python/FormulaColumnPython.json @@ -4,7 +4,9 @@ "copy": "Create a copy of this SelectColumn.\n\n:return: (io.deephaven.engine.table.impl.select.SelectColumn) an independent copy of this SelectColumn.", "create": ":param columnName: java.lang.String\n:param dcf: io.deephaven.engine.table.impl.select.python.DeephavenCompatibleFunction\n:return: io.deephaven.engine.table.impl.select.python.FormulaColumnPython", "createInstance": ":param arrays: io.deephaven.vector.Vector[]\n:param params: io.deephaven.engine.table.lang.QueryScopeParam[]\n:return: io.deephaven.engine.table.impl.select.formula.FormulaKernel", - "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnNameMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent" + "initDef": "Initialize any internal column definitions from the provided initial.\n\n:param columnNameMap: (java.util.Map>) - the starting set of column definitions\n:return: (java.util.List) a list of columns on which the result of this is dependent", + "isStateless": "Returns true if this column is stateless (i.e. one row does not depend on the order of evaluation for another\n row).\n\n:return: boolean", + "preventsParallelization": ":return: boolean" }, "path": "io.deephaven.engine.table.impl.select.python.FormulaColumnPython", "text": "A formula column for python native code.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/AbstractSparseLongArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/AbstractSparseLongArraySource.json index 6081af429da..d549f953dcf 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/AbstractSparseLongArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/AbstractSparseLongArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.AbstractSparseLongArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "getChunk": "Returns a chunk of data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.GetContext) - A context containing all mutable/state related data used in retrieving the Chunk. In particular,\n the Context may be used to provide a Chunk data pool\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched\n:return: (io.deephaven.chunk.Chunk) A chunk of data corresponding to the keys from the given RowSequence", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitMaskingColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitMaskingColumnSource.json index 8544eb71919..5e3458bf230 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitMaskingColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitMaskingColumnSource.json @@ -44,8 +44,10 @@ "getUngroupedShort": ":param columnIndex: long\n:param arrayIndex: int\n:return: short", "getUngroupedSize": ":param columnIndex: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.impl.sources.BitMaskingColumnSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "releaseCachedResources": "Release any resources held for caching purposes. Implementations need not guarantee that concurrent accesses are\n correct, as the purpose of this method is to ensure cleanup for column sources that will no longer be used.", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitShiftingColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitShiftingColumnSource.json index a214d5fe79d..2aa23f44531 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitShiftingColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BitShiftingColumnSource.json @@ -44,8 +44,10 @@ "getUngroupedShort": ":param columnIndex: long\n:param arrayIndex: int\n:return: short", "getUngroupedSize": ":param columnIndex: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.impl.sources.BitShiftingColumnSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "releaseCachedResources": "Release any resources held for caching purposes. Implementations need not guarantee that concurrent accesses are\n correct, as the purpose of this method is to ensure cleanup for column sources that will no longer be used.", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanAsByteColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanAsByteColumnSource.json index e7f9755f28a..1bcdb19012b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanAsByteColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanAsByteColumnSource.json @@ -8,7 +8,9 @@ "getByte": ":param index: long\n:return: byte", "getPrevByte": ":param index: long\n:return: byte", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", - "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations" + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.BooleanAsByteColumnSource", "text": "Reinterpret result ColumnSource implementations that translates Boolean to byte values.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.json index 009e7f3347e..f10c0fa3de0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BooleanSparseArraySource.json @@ -3,6 +3,7 @@ "methods": { "allowsReinterpret": "Test if a reinterpret call will succeed.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternative type to consider\n:return: (boolean) If a reinterpret on this column source with the supplied alternateDataType will succeed.", "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Boolean", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BoxedColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BoxedColumnSource.json index 04d273183da..1fb844f8f4e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BoxedColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/BoxedColumnSource.json @@ -7,7 +7,9 @@ "get": ":param index: long\n:return: BoxedColumnSource.DATA_TYPE", "getPrev": ":param index: long\n:return: BoxedColumnSource.DATA_TYPE", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", - "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations" + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.BoxedColumnSource", "text": "ColumnSource implementation for explicitly boxing a primitive into a more complex type, e.g. byte as\n Boolean or long as DateTime.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteAsBooleanColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteAsBooleanColumnSource.json index d4faf1ae9df..93075e6974d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteAsBooleanColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteAsBooleanColumnSource.json @@ -10,7 +10,9 @@ "get": ":param index: long\n:return: java.lang.Boolean", "getPrev": ":param index: long\n:return: java.lang.Boolean", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "providesFillUnordered": "Returns true if this column source can efficiently provide an unordered fill.\n\n If this method returns false, then fillChunkUnordered and fillPrevChunkUnordered may throw an\n UnsupportedOperationException.\n\n:return: (boolean) if this column source can provide an unordered fill" }, "path": "io.deephaven.engine.table.impl.sources.ByteAsBooleanColumnSource", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.json index 930bd3cbdd4..1eb6312e96d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ByteSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.ByteSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Byte", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CharacterSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CharacterSparseArraySource.json index 8287cd0db68..675d70a24d8 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CharacterSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CharacterSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.CharacterSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Character", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CrossJoinRightColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CrossJoinRightColumnSource.json index 07292b4d3a0..892cfac6a6c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CrossJoinRightColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/CrossJoinRightColumnSource.json @@ -44,8 +44,10 @@ "getUngroupedShort": ":param columnIndex: long\n:param arrayIndex: int\n:return: short", "getUngroupedSize": ":param columnIndex: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.impl.sources.CrossJoinRightColumnSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "releaseCachedResources": "Release any resources held for caching purposes. Implementations need not guarantee that concurrent accesses are\n correct, as the purpose of this method is to ensure cleanup for column sources that will no longer be used.", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DateTimeAsLongColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DateTimeAsLongColumnSource.json index 9f62a2a98eb..4fd320562e5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DateTimeAsLongColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DateTimeAsLongColumnSource.json @@ -8,7 +8,9 @@ "getLong": ":param index: long\n:return: long", "getPrevLong": ":param index: long\n:return: long", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", - "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations" + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.DateTimeAsLongColumnSource", "text": "Reinterpret result ColumnSource implementations that translates DateTime to long values.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.json index ea4a9a17569..d4e97f6937a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DelegatingColumnSource.json @@ -25,9 +25,11 @@ "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "**Incompatible overloads text - text from the first overload:**\n\nAllocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource, without a SharedContext.\n\n*Overload 1* \n :param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n :param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n :return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations\n \n*Overload 2* \n :param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n :return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", "makeGetContext": "**Incompatible overloads text - text from the first overload:**\n\nAllocate a new ChunkSource.GetContext for retrieving chunks from this FillContextMaker, typically\n a ChunkSource without a SharedContext.\n\n*Overload 1* \n :param chunkCapacity: (int) - The maximum size required for any WritableChunk allocated as part of the result.\n :param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n :return: (io.deephaven.engine.table.ChunkSource.GetContext) A context for use with get operations\n \n*Overload 2* \n :param chunkCapacity: (int) - The maximum size required for any WritableChunk allocated as part of the result.\n :return: (io.deephaven.engine.table.ChunkSource.GetContext) A context for use with get operations", "match": ":param invertMatch: boolean\n:param usePrev: boolean\n:param caseInsensitive: boolean\n:param mapper: io.deephaven.engine.rowset.RowSet\n:param keys: java.lang.Object...\n:return: io.deephaven.engine.rowset.WritableRowSet", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "releaseCachedResources": "Release any resources held for caching purposes. Implementations need not guarantee that concurrent accesses are\n correct, as the purpose of this method is to ensure cleanup for column sources that will no longer be used.", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.json index 9242aada6b1..3571139fe1d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/DoubleSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.DoubleSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Double", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.json index 6d990b737ac..2ff4d2f5e5f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/FloatSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.FloatSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Float", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.json index 9ca6b038e8f..f6f56e4d3b5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/IntegerSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.IntegerSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Integer", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/LongAsDateTimeColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/LongAsDateTimeColumnSource.json index 411ff3e2868..22f8e386d8d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/LongAsDateTimeColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/LongAsDateTimeColumnSource.json @@ -10,7 +10,9 @@ "get": ":param index: long\n:return: io.deephaven.time.DateTime", "getPrev": ":param index: long\n:return: io.deephaven.time.DateTime", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "providesFillUnordered": "Returns true if this column source can efficiently provide an unordered fill.\n\n If this method returns false, then fillChunkUnordered and fillPrevChunkUnordered may throw an\n UnsupportedOperationException.\n\n:return: (boolean) if this column source can provide an unordered fill" }, "path": "io.deephaven.engine.table.impl.sources.LongAsDateTimeColumnSource", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ObjectSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ObjectSparseArraySource.json index cd5fcd4f6fb..8f5a24222d7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ObjectSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ObjectSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.ObjectSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: ObjectSparseArraySource.T", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.json index 496afa87676..f485ee144e7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/RedirectedColumnSource.json @@ -45,8 +45,10 @@ "getUngroupedShort": ":param columnIndex: long\n:param arrayIndex: int\n:return: short", "getUngroupedSize": ":param columnIndex: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.impl.sources.RedirectedColumnSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "releaseCachedResources": "Release any resources held for caching purposes. Implementations need not guarantee that concurrent accesses are\n correct, as the purpose of this method is to ensure cleanup for column sources that will no longer be used.", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ReversedColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ReversedColumnSource.json index b195a83c123..b1f5dcb2788 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ReversedColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ReversedColumnSource.json @@ -23,7 +23,9 @@ "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.impl.sources.ReversedColumnSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, "path": "io.deephaven.engine.table.impl.sources.ReversedColumnSource", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.json index 40f7fe0e194..7a4d625be14 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ShortSparseArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.ShortSparseArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFill: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param changedRows: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillFromChunkUnordered": "Fills the ChunkSink with data from the source, with data corresponding to the keys from the given key chunk.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data RowSequence\n:param keys: (io.deephaven.chunk.LongChunk) - A LongChunk representing the keys to be written", "fillPrevChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made\n about the size of the chunk shall be made. The chunk will be populated from position\n [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "get": ":param index: long\n:return: java.lang.Short", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/SwitchColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/SwitchColumnSource.json index c018fe69fb6..e3cb57608ca 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/SwitchColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/SwitchColumnSource.json @@ -24,8 +24,10 @@ "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", "makeGetContext": "Allocate a new ChunkSource.GetContext for retrieving chunks from this GetContextMaker, typically\n a ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size required for any WritableChunk allocated as part of the result.\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.GetContext) A context for use with get operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "setNewCurrent": ":param newCurrent: io.deephaven.engine.table.ColumnSource" }, "path": "io.deephaven.engine.table.impl.sources.SwitchColumnSource", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteArrayColumnSource.json index 26daa6e8516..2c050612110 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteArrayColumnSource.json @@ -4,7 +4,9 @@ "getByte": ":param index: long\n:return: byte", "getComponentType": ":return: java.lang.Class", "getPrevByte": ":param index: long\n:return: byte", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedByteArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteVectorColumnSource.json index 274d54af982..a5b9fa76e6f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedByteVectorColumnSource.json @@ -6,7 +6,9 @@ "getComponentType": ":return: java.lang.Class", "getPrev": ":param index: long\n:return: java.lang.Byte", "getPrevByte": ":param index: long\n:return: byte", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedByteVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharArrayColumnSource.json index 50d3ad180c7..69645af8953 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharArrayColumnSource.json @@ -4,7 +4,9 @@ "getChar": ":param index: long\n:return: char", "getComponentType": ":return: java.lang.Class", "getPrevChar": ":param index: long\n:return: char", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedCharArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharVectorColumnSource.json index 3f53cd1f24e..528f294d10a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedCharVectorColumnSource.json @@ -6,7 +6,9 @@ "getComponentType": ":return: java.lang.Class", "getPrev": ":param index: long\n:return: java.lang.Character", "getPrevChar": ":param index: long\n:return: char", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedCharVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleArrayColumnSource.json index de6ebbd68ed..4de036b75fa 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleArrayColumnSource.json @@ -4,7 +4,9 @@ "getComponentType": ":return: java.lang.Class", "getDouble": ":param index: long\n:return: double", "getPrevDouble": ":param index: long\n:return: double", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedDoubleArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleVectorColumnSource.json index 5a1961c19a1..1dc5cf122d0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedDoubleVectorColumnSource.json @@ -6,7 +6,9 @@ "getDouble": ":param index: long\n:return: double", "getPrev": ":param index: long\n:return: java.lang.Double", "getPrevDouble": ":param index: long\n:return: double", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedDoubleVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatArrayColumnSource.json index 7a1d4df9242..be16705a58b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatArrayColumnSource.json @@ -4,7 +4,9 @@ "getComponentType": ":return: java.lang.Class", "getFloat": ":param index: long\n:return: float", "getPrevFloat": ":param index: long\n:return: float", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedFloatArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatVectorColumnSource.json index b1992ce0c47..58e00c94594 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedFloatVectorColumnSource.json @@ -6,7 +6,9 @@ "getFloat": ":param index: long\n:return: float", "getPrev": ":param index: long\n:return: java.lang.Float", "getPrevFloat": ":param index: long\n:return: float", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedFloatVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntArrayColumnSource.json index ea1aed9032e..7deb1e504c7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntArrayColumnSource.json @@ -4,7 +4,9 @@ "getComponentType": ":return: java.lang.Class", "getInt": ":param index: long\n:return: int", "getPrevInt": ":param index: long\n:return: int", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedIntArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntVectorColumnSource.json index 19303157382..2828c451e95 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedIntVectorColumnSource.json @@ -6,7 +6,9 @@ "getInt": ":param index: long\n:return: int", "getPrev": ":param index: long\n:return: java.lang.Integer", "getPrevInt": ":param index: long\n:return: int", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedIntVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongArrayColumnSource.json index 4830d6f4085..f0abed26c67 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongArrayColumnSource.json @@ -4,7 +4,9 @@ "getComponentType": ":return: java.lang.Class", "getLong": ":param index: long\n:return: long", "getPrevLong": ":param index: long\n:return: long", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedLongArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongVectorColumnSource.json index b27fcbdeecd..0f01005635b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedLongVectorColumnSource.json @@ -6,7 +6,9 @@ "getLong": ":param index: long\n:return: long", "getPrev": ":param index: long\n:return: java.lang.Long", "getPrevLong": ":param index: long\n:return: long", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedLongVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortArrayColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortArrayColumnSource.json index d95b0f71dab..2c3d04fe856 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortArrayColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortArrayColumnSource.json @@ -4,7 +4,9 @@ "getComponentType": ":return: java.lang.Class", "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedShortArrayColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortVectorColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortVectorColumnSource.json index 1ed77169380..785cb38b0a4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortVectorColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UngroupedShortVectorColumnSource.json @@ -6,7 +6,9 @@ "getPrev": ":param index: long\n:return: java.lang.Short", "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UngroupedShortVectorColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UnionColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UnionColumnSource.json index f13ae5a2775..664aaaa676c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UnionColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/UnionColumnSource.json @@ -25,7 +25,9 @@ "getSubSource": ":param i: int\n:return: io.deephaven.engine.table.ColumnSource", "getUnionSourceManager": "Return the Union source manager that was used to create this table.\n\n:return: io.deephaven.engine.table.impl.sources.UnionSourceManager", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", - "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations" + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.sources.UnionColumnSource", "text": "ColumnSource implementation for TableTools.merge(java.util.List) results.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ViewColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ViewColumnSource.json index e5ae24eeb8f..bd1dbea6c59 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ViewColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/ViewColumnSource.json @@ -24,8 +24,10 @@ "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedState: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", "makeGetContext": "Allocate a new ChunkSource.GetContext for retrieving chunks from this GetContextMaker, typically\n a ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size required for any WritableChunk allocated as part of the result.\n:param sharedState: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.GetContext) A context for use with get operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op." }, "path": "io.deephaven.engine.table.impl.sources.ViewColumnSource", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ByteAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ByteAggregateColumnSource.json index 408589613ae..ebef40af902 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ByteAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ByteAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/CharAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/CharAggregateColumnSource.json index 3d486941cd7..ec017c0af2b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/CharAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/CharAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/DoubleAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/DoubleAggregateColumnSource.json index d675aaddc4b..c85b3378d84 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/DoubleAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/DoubleAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/FloatAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/FloatAggregateColumnSource.json index 5b9943f72f6..95f82330fc4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/FloatAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/FloatAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/IntAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/IntAggregateColumnSource.json index 0acf9c7c742..c98ec7b0e39 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/IntAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/IntAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/LongAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/LongAggregateColumnSource.json index 79f39bbe9f2..a32b332d113 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/LongAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/LongAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ObjectAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ObjectAggregateColumnSource.json index 47ac26c0f28..dc73a41568d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ObjectAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ObjectAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ShortAggregateColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ShortAggregateColumnSource.json index b42c8c02e1d..d10d0f3de0f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ShortAggregateColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/aggregate/ShortAggregateColumnSource.json @@ -25,8 +25,10 @@ "getUngroupedPrevSize": ":param groupIndexKey: long\n:return: long", "getUngroupedShort": ":param groupIndexKey: long\n:param offsetInGroup: int\n:return: short", "getUngroupedSize": ":param groupIndexKey: (long) - the row key within this column to interrogate\n:return: (long) the size of the Vector at columnIndex.", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", "isUngroupable": "Does this particular instance of the column source support ungrouping?\n\n:return: (boolean) true if you can call the getUngrouped family of methods and get a valid answer.", "makeFillContext": "Allocate a new ChunkSource.FillContext for filling chunks from this FillContextMaker, typically a\n ChunkSource.\n\n:param chunkCapacity: (int) - The maximum size of any WritableChunk that will be filled with this context\n:param sharedContext: (io.deephaven.engine.table.SharedContext) - Shared store of intermediate results.\n:return: (io.deephaven.engine.table.ChunkSource.FillContext) A context for use with fill operations", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization", "startTrackingPrevValues": "ColumnSource implementations that track previous values have the option to not actually start tracking previous\n values until this method is called. This is an option, not an obligation: some simple ColumnSource\n implementations (like TSingleValueSource for various T) always track previous values; other implementations (like\n PrevColumnSource) never do; some (like TArrayColumnSource) only start tracking once this method is called.\n\n An immutable column source can not have distinct prev values; therefore it is implemented as a no-op.", "ungrouped": ":return: io.deephaven.engine.table.impl.sources.UngroupedColumnSource" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.json index b6369df9ebd..9b46777d175 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DByteArraySource.json @@ -3,6 +3,7 @@ "methods": { "allowsReinterpret": "Test if a reinterpret call will succeed.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternative type to consider\n:return: (boolean) If a reinterpret on this column source with the supplied alternateDataType will succeed.", "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.json index 7ada62452d9..ed3e8e7e72a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DCharArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.Immutable2DCharArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.json index ed2b51553eb..d6e526c8606 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DDoubleArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.Immutable2DDoubleArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.json index bf11206143b..b37ce96f1ea 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DFloatArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.Immutable2DFloatArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.json index 794a46c947e..2db66c44991 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DIntArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.Immutable2DIntArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.json index c1bc5aa32bd..07901149fe1 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DLongArraySource.json @@ -3,6 +3,7 @@ "methods": { "allowsReinterpret": "Test if a reinterpret call will succeed.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternative type to consider\n:return: (boolean) If a reinterpret on this column source with the supplied alternateDataType will succeed.", "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.json index 9f11ea98626..db3756b6baf 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DObjectArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.Immutable2DObjectArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.json index c5a1adb74fa..c81683ef8ca 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/Immutable2DShortArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.Immutable2DShortArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.json index d2101a4acc8..9d5fb93c8db 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableByteArraySource.json @@ -3,6 +3,7 @@ "methods": { "allowsReinterpret": "Test if a reinterpret call will succeed.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternative type to consider\n:return: (boolean) If a reinterpret on this column source with the supplied alternateDataType will succeed.", "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.json index 5723afce087..1b0e96864fa 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableCharArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableCharArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.json index 98f43a6d471..46a20182aa9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableDoubleArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableDoubleArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.json index ffdd789f4ba..49b40c193f2 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableFloatArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableFloatArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.json index eeac6e53af0..f644979a6c3 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableIntArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableIntArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.json index 7987099ae72..da3e9bb1242 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableLongArraySource.json @@ -3,6 +3,7 @@ "methods": { "allowsReinterpret": "Test if a reinterpret call will succeed.\n\nNote: Java generics information - \n\n:param alternateDataType: (java.lang.Class) - The alternative type to consider\n:return: (boolean) If a reinterpret on this column source with the supplied alternateDataType will succeed.", "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.json index 6aeb0fec979..d61049612c1 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableObjectArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableObjectArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.json index e07da6e4925..32134a73a4d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/sources/immutable/ImmutableShortArraySource.json @@ -2,6 +2,7 @@ "className": "io.deephaven.engine.table.impl.sources.immutable.ImmutableShortArraySource", "methods": { "ensureCapacity": "Ensure that this WritableColumnSource can accept row keys in range [0, capacity).\n\n:param capacity: (long) - The new minimum capacity\n:param nullFilled: (boolean) - Whether data should be \"null-filled\". If true, get operations at row keys that have not been\n set will return the appropriate null value; otherwise such gets produce undefined results.", + "ensurePrevious": "Prepare this column source such that all values in rowSet may be accessed using getPrev. Further operations in\n this cycle need not check for previous when writing data to the column source; you must provide a row set that\n contains every row that may be written to this column source.\n\n:param rowSet: (io.deephaven.engine.rowset.RowSet) - the rowset of values that will change on this cycle", "fillChunk": "Populates the given destination chunk with data corresponding to the keys from the given RowSequence.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param destination: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to rowSequence. No assumptions shall be made about\n the size of the chunk shall be made. The chunk will be populated from position [0,rowSequence.size()).\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be fetched", "fillChunkUnordered": "Populates a contiguous portion of the given destination chunk with data corresponding to the keys from the given\n LongChunk.\n\n:param context: (io.deephaven.engine.table.ChunkSource.FillContext) - A context containing all mutable/state related data used in retrieving the Chunk.\n:param dest: (io.deephaven.chunk.WritableChunk) - The chunk to be populated according to keys\n:param keys: (io.deephaven.chunk.LongChunk) - A chunk of individual, not assumed to be ordered keys to be fetched", "fillFromChunk": "Our default, inefficient, implementation. Inheritors who care should provide a better implementation.\n\n:param context: (io.deephaven.engine.table.ChunkSink.FillFromContext) - A context containing all mutable/state related data used in writing the Chunk.\n:param src: (io.deephaven.chunk.Chunk) - The source of the data rowSequence\n:param rowSequence: (io.deephaven.engine.rowset.RowSequence) - An RowSequence representing the keys to be written", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/AppendOnlyArrayBackedMutableTable.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/AppendOnlyArrayBackedMutableTable.json index c6d0d274f9b..155ca2880b7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/AppendOnlyArrayBackedMutableTable.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/AppendOnlyArrayBackedMutableTable.json @@ -2,6 +2,8 @@ "className": "io.deephaven.engine.table.impl.util.AppendOnlyArrayBackedMutableTable", "methods": { "make": "**Incompatible overloads text - text from the first overload:**\n\nCreate an AppendOnlyArrayBackedMutableTable with the given initial data.\n\n*Overload 1* \n :param definition: (io.deephaven.engine.table.TableDefinition) - the definition of the new table.\n :return: (io.deephaven.engine.table.impl.util.AppendOnlyArrayBackedMutableTable) an empty AppendOnlyArrayBackedMutableTable with the given definition\n \n*Overload 2* \n :param definition: (io.deephaven.engine.table.TableDefinition) - the definition of the new table.\n :param enumValues: (java.util.Map) - a map of column names to enumeration values\n :return: (io.deephaven.engine.table.impl.util.AppendOnlyArrayBackedMutableTable) an empty AppendOnlyArrayBackedMutableTable with the given definition\n \n*Overload 3* \n :param initialTable: (io.deephaven.engine.table.Table) - the initial values to copy into the AppendOnlyArrayBackedMutableTable\n :return: (io.deephaven.engine.table.impl.util.AppendOnlyArrayBackedMutableTable) an empty AppendOnlyArrayBackedMutableTable with the given definition\n \n*Overload 4* \n :param initialTable: (io.deephaven.engine.table.Table) - the initial values to copy into the AppendOnlyArrayBackedMutableTable\n :param enumValues: (java.util.Map) - a map of column names to enumeration values\n :return: (io.deephaven.engine.table.impl.util.AppendOnlyArrayBackedMutableTable) an empty AppendOnlyArrayBackedMutableTable with the given definition", + "mutableInputTable": ":return: io.deephaven.engine.util.config.MutableInputTable", + "readOnlyCopy": ":return: io.deephaven.engine.table.Table", "setDescription": ":param newDescription: java.lang.String\n:return: io.deephaven.engine.table.impl.util.BaseArrayBackedMutableTable" }, "path": "io.deephaven.engine.table.impl.util.AppendOnlyArrayBackedMutableTable", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/KeyedArrayBackedMutableTable.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/KeyedArrayBackedMutableTable.json index fad31b9c4cf..8dda15109f2 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/KeyedArrayBackedMutableTable.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/KeyedArrayBackedMutableTable.json @@ -2,6 +2,8 @@ "className": "io.deephaven.engine.table.impl.util.KeyedArrayBackedMutableTable", "methods": { "make": "**Incompatible overloads text - text from the first overload:**\n\nCreate an empty KeyedArrayBackedMutableTable.\n\n*Overload 1* \n :param definition: (io.deephaven.engine.table.TableDefinition) - the definition of the table to create\n :param keyColumnNames: (java.lang.String...) - the name of the key columns\n :return: (io.deephaven.engine.table.impl.util.KeyedArrayBackedMutableTable) an empty KeyedArrayBackedMutableTable with the given definition and key columns\n \n*Overload 2* \n :param definition: (io.deephaven.engine.table.TableDefinition) - the definition of the table to create\n :param enumValues: (java.util.Map) - a map of column names to enumeration values\n :param keyColumnNames: (java.lang.String...) - the name of the key columns\n :return: (io.deephaven.engine.table.impl.util.KeyedArrayBackedMutableTable) an empty KeyedArrayBackedMutableTable with the given definition and key columns\n \n*Overload 3* \n :param initialTable: (io.deephaven.engine.table.Table) - the initial values to copy into the KeyedArrayBackedMutableTable\n :param keyColumnNames: (java.lang.String...) - the name of the key columns\n :return: (io.deephaven.engine.table.impl.util.KeyedArrayBackedMutableTable) an empty KeyedArrayBackedMutableTable with the given definition and key columns\n \n*Overload 4* \n :param initialTable: (io.deephaven.engine.table.Table) - the initial values to copy into the KeyedArrayBackedMutableTable\n :param enumValues: (java.util.Map) - a map of column names to enumeration values\n :param keyColumnNames: (java.lang.String...) - the name of the key columns\n :return: (io.deephaven.engine.table.impl.util.KeyedArrayBackedMutableTable) an empty KeyedArrayBackedMutableTable with the given definition and key columns", + "mutableInputTable": ":return: io.deephaven.engine.util.config.MutableInputTable", + "readOnlyCopy": ":return: io.deephaven.engine.table.Table", "setDescription": ":param newDescription: java.lang.String\n:return: io.deephaven.engine.table.impl.util.BaseArrayBackedMutableTable" }, "path": "io.deephaven.engine.table.impl.util.KeyedArrayBackedMutableTable", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/MergeSortedHelper/SortedMergeColumnSource.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/MergeSortedHelper/SortedMergeColumnSource.json index fab675a2734..d930b149a32 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/MergeSortedHelper/SortedMergeColumnSource.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/util/MergeSortedHelper/SortedMergeColumnSource.json @@ -20,7 +20,9 @@ "getPrevLong": ":param index: long\n:return: long", "getPrevShort": ":param index: long\n:return: short", "getShort": ":param index: long\n:return: short", - "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise" + "isImmutable": "Determine if this column source is immutable, meaning that the values at a given row key never change.\n\n:return: (boolean) true if the values at a given row key of the column source never change, false otherwise", + "isStateless": "Most column sources will return the same value for a given row without respect to the order that the rows are\n read. Those columns sources are considered \"stateless\" and should return true.\n\n Some column sources, however may be dependent on evaluation order. For example, a formula that updates a Map must\n be evaluated from the first row to the last row. A column source that has the potential to depend on the order of\n evaluation must return false.\n\n:return: (boolean) true if this is a stateless column source", + "preventsParallelism": "Can this column source be evaluated on an arbitrary thread?\n\n Most column sources can be evaluated on an arbitrary thread, however those that do call into Python can not be\n evaluated on an arbitrary thread as the calling thread may already have the GIL, which would result in a deadlock\n when the column source takes the GIL to evaluate formulas.\n\n:return: (boolean) true if this column prevents parallelization" }, "path": "io.deephaven.engine.table.impl.util.MergeSortedHelper.SortedMergeColumnSource", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils.json new file mode 100644 index 00000000000..097bc8b8e63 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.engine.util.BigDecimalUtils", + "methods": { + "computePrecisionAndScale": "**Incompatible overloads text - text from the first overload:**\n\nCompute an overall precision and scale that would fit all existing values in a column source.\n\n*Overload 1* \n :param t: (io.deephaven.engine.table.Table) - a Deephaven table\n :param colName: (java.lang.String) - a Column for t, which should be of BigDecimal type\n :return: (io.deephaven.engine.util.BigDecimalUtils.PrecisionAndScale) a PrecisionAndScale object result.\n \n*Overload 2* \n :param rowSet: (io.deephaven.engine.rowset.TrackingRowSet) - The rowset for the provided column\n :param source: (io.deephaven.engine.table.ColumnSource) - a ColumnSource of BigDecimal type\n :return: (io.deephaven.engine.util.BigDecimalUtils.PrecisionAndScale) a PrecisionAndScale object result.", + "getPrecisionAndScaleFromColumnProperties": "Get a PrecisionAndScale value from a object.\n\n:param propertyNames: (io.deephaven.engine.util.BigDecimalUtils.PropertyNames) - The property names to read.\n:param columnProperties: (java.util.Properties) - The object from where to read the properties\n:param allowNulls: (boolean) - If true, do not throw when a property is missing, instead set the value to\n \n:return: (io.deephaven.engine.util.BigDecimalUtils.PrecisionAndScale) A object with the values read.", + "setProperties": "Set the given names and values in the supplied Properties object.\n\n:param props: (java.util.Properties) - Properties where the given property names and values would be set.\n:param names: (io.deephaven.engine.util.BigDecimalUtils.PropertyNames) - Property names to set\n:param values: (io.deephaven.engine.util.BigDecimalUtils.PrecisionAndScale) - Property values to set" + }, + "path": "io.deephaven.engine.util.BigDecimalUtils", + "text": "Utilities to support BigDecimal exhaust.\n\n Parquet and Avro decimal types make a whole column decimal type have a fixed precision and scale; BigDecimal columns\n in Deephaven are, each value, arbitrary precision (its own precision and scale).\n\n For static tables, it is possible to compute overall precision and scale values that fit every existing value. For\n refreshing tables, we need the user to tell us.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PrecisionAndScale.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PrecisionAndScale.json new file mode 100644 index 00000000000..15daeb26d20 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PrecisionAndScale.json @@ -0,0 +1,7 @@ +{ + "className": "io.deephaven.engine.util.BigDecimalUtils$PrecisionAndScale", + "methods": {}, + "path": "io.deephaven.engine.util.BigDecimalUtils.PrecisionAndScale", + "text": "Immutable way to store and pass precision and scale values.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PropertyNames.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PropertyNames.json new file mode 100644 index 00000000000..05389136cdf --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/BigDecimalUtils/PropertyNames.json @@ -0,0 +1,7 @@ +{ + "className": "io.deephaven.engine.util.BigDecimalUtils$PropertyNames", + "methods": {}, + "path": "io.deephaven.engine.util.BigDecimalUtils.PropertyNames", + "text": "Immutable way to store and pass properties to get precision and scale for a given named column.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/ExportedObjectType.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/ExportedObjectType.json deleted file mode 100644 index 99a7f072690..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/util/ExportedObjectType.json +++ /dev/null @@ -1,13 +0,0 @@ -{ - "className": "io.deephaven.engine.util.ExportedObjectType", - "methods": { - "fromObject": ":param value: java.lang.Object\n:return: io.deephaven.engine.util.ExportedObjectType", - "isDisplayable": ":return: boolean", - "isTableType": ":return: boolean", - "isWidgetType": ":return: boolean", - "valueOf": "Returns the enum constant of this type with the specified name.\nThe string must match exactly an identifier used to declare an\nenum constant in this type. (Extraneous whitespace characters are \nnot permitted.)\n\n:param name: (java.lang.String) - the name of the enum constant to be returned.\n:return: (io.deephaven.engine.util.ExportedObjectType) the enum constant with the specified name", - "values": "Returns an array containing the constants of this enum type, in\nthe order they are declared.\n\n:return: (io.deephaven.engine.util.ExportedObjectType[]) an array containing the constants of this enum type, in the order they are declared" - }, - "path": "io.deephaven.engine.util.ExportedObjectType", - "typeName": "enum" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/GroovyDeephavenSession/GroovySnapshot.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/GroovyDeephavenSession/GroovySnapshot.json new file mode 100644 index 00000000000..7d099205902 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/GroovyDeephavenSession/GroovySnapshot.json @@ -0,0 +1,6 @@ +{ + "className": "io.deephaven.engine.util.GroovyDeephavenSession$GroovySnapshot", + "methods": {}, + "path": "io.deephaven.engine.util.GroovyDeephavenSession.GroovySnapshot", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/IsWidget.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/IsWidget.json deleted file mode 100644 index 0cc06be1422..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/util/IsWidget.json +++ /dev/null @@ -1,14 +0,0 @@ -{ - "className": "io.deephaven.engine.util.IsWidget", - "methods": { - "getLiveWidgetVisibility": ":param object: java.lang.Object\n:return: java.lang.String[]", - "getTable": "*Overload 1* \n :param value: java.lang.Object\n :return: io.deephaven.engine.table.Table\n \n*Overload 2* \n :param pyObject: org.jpy.PyObject\n :return: io.deephaven.engine.table.Table", - "getWidget": "*Overload 1* \n :param value: java.lang.Object\n :return: io.deephaven.engine.util.LiveWidget\n \n*Overload 2* \n :param pyObject: org.jpy.PyObject\n :return: io.deephaven.engine.util.LiveWidget", - "isLiveWidgetVisibilityProvider": ":param value: java.lang.Object\n:return: boolean", - "isTable": ":param value: java.lang.Object\n:return: boolean", - "isWidget": ":param value: java.lang.Object\n:return: boolean" - }, - "path": "io.deephaven.engine.util.IsWidget", - "text": "Get a widget from an object.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/util/PythonDeephavenSession/PythonSnapshot.json b/Integrations/python/deephaven/doc/io/deephaven/engine/util/PythonDeephavenSession/PythonSnapshot.json new file mode 100644 index 00000000000..0885db37d41 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/util/PythonDeephavenSession/PythonSnapshot.json @@ -0,0 +1,6 @@ +{ + "className": "io.deephaven.engine.util.PythonDeephavenSession$PythonSnapshot", + "methods": {}, + "path": "io.deephaven.engine.util.PythonDeephavenSession.PythonSnapshot", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetPluginRegistration.json b/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetPluginRegistration.json new file mode 100644 index 00000000000..b94f8d2efc2 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetPluginRegistration.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.figure.FigureWidgetPluginRegistration", + "methods": { + "registerInto": "The registration entrypoint.\n\n \n May be called multiple times.\n\n:param callback: (io.deephaven.plugin.Registration.Callback) - the callback." + }, + "path": "io.deephaven.figure.FigureWidgetPluginRegistration", + "text": "Registers the object type plugin FigureWidgetTypePlugin.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTranslator.json b/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTranslator.json new file mode 100644 index 00000000000..c69f0f92d6d --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTranslator.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.figure.FigureWidgetTranslator", + "methods": { + "translate": ":param figure: io.deephaven.plot.FigureWidget\n:param exporter: io.deephaven.plugin.type.ObjectType.Exporter\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor" + }, + "path": "io.deephaven.figure.FigureWidgetTranslator", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTypePlugin.json b/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTypePlugin.json new file mode 100644 index 00000000000..24611a2d35a --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/figure/FigureWidgetTypePlugin.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.figure.FigureWidgetTypePlugin", + "methods": { + "instance": ":return: io.deephaven.figure.FigureWidgetTypePlugin", + "writeToImpl": ":param exporter: io.deephaven.plugin.type.ObjectType.Exporter\n:param figureWidget: io.deephaven.plot.FigureWidget\n:param out: java.io.OutputStream" + }, + "path": "io.deephaven.figure.FigureWidgetTypePlugin", + "text": "An object type named \"Figure\" of java class type FigureWidget.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools/Produce.json b/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools/Produce.json index c343b3e3664..922a6c513c3 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools/Produce.json +++ b/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools/Produce.json @@ -1,9 +1,9 @@ { "className": "io.deephaven.kafka.KafkaTools$Produce", "methods": { - "avroSpec": "**Incompatible overloads text - text from the first overload:**\n\nAvro spec from fetching an Avro schema from a Confluent compatible Schema Server. The Properties used to\n initialize Kafka should contain the URL for the Schema Server to use under the \"schema.registry.url\"\n property.\n\n*Overload 1* \n :param schema: (org.apache.avro.Schema) - An Avro schema. The message will implement this schema; all fields will be populated from some\n table column via explicit or implicit mapping.\n :param fieldToColumnMapping: (java.util.Map) - A map from Avro schema field name to column name. Any field names not included as\n a key in the map will be mapped to columns with the same name. If null, map all fields to columns of\n the same name.\n :param timestampFieldName: (java.lang.String) - If not null, include a field of the given name with a publication timestamp. The\n field with the given name should exist in the provided schema, and be of logical type timestamp\n micros.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A spec corresponding to the schema provided.\n \n*Overload 2* \n :param schemaName: (java.lang.String) - The registered name for the schema on Schema Server\n :param schemaVersion: (java.lang.String) - The version to fetch. Pass the constant AVRO_LATEST_VERSION for latest\n :param fieldToColumnMapping: (java.util.Map) - A map from Avro schema field name to column name. Any field names not included as\n a key in the map will be mapped to columns with the same name. If null, map all fields to columns of\n the same name.\n :param timestampFieldName: (java.lang.String) - If not null, include a field of the given name with a publication timestamp. The\n field with the given name should exist in the provided schema, and be of logical type timestamp\n micros.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A spec corresponding to the schema provided.", + "avroSpec": "**Incompatible overloads text - text from the first overload:**\n\nAvro spec from fetching an Avro schema from a Confluent compatible Schema Server. The Properties used to\n initialize Kafka should contain the URL for the Schema Server to use under the \"schema.registry.url\"\n property.\n\n*Overload 1* \n :param schema: (org.apache.avro.Schema) - An Avro schema. The message will implement this schema; all fields will be populated from some\n table column via explicit or implicit mapping.\n :param fieldToColumnMapping: (java.util.Map) - A map from Avro schema field name to column name. Any field names not included as\n a key in the map will be mapped to columns with the same name (unless those columns are filtered out).\n If null, map all fields to columns of the same name (except for columns filtered out).\n :param timestampFieldName: (java.lang.String) - If not null, include a field of the given name with a publication timestamp. The\n field with the given name should exist in the provided schema, and be of logical type timestamp\n micros.\n :param includeOnlyColumns: (java.util.function.Predicate) - If not null, filter out any columns tested false in this predicate.\n :param excludeColumns: (java.util.function.Predicate) - If not null, filter out any columns tested true in this predicate.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A spec corresponding to the schema provided.\n \n*Overload 2* \n :param schemaName: (java.lang.String) - The registered name for the schema on Schema Server\n :param schemaVersion: (java.lang.String) - The version to fetch. Pass the constant AVRO_LATEST_VERSION for latest\n :param fieldToColumnMapping: (java.util.Map) - A map from Avro schema field name to column name. Any field names not included as\n a key in the map will be mapped to columns with the same name. If null, map all fields to columns of\n the same name.\n :param timestampFieldName: (java.lang.String) - If not null, include a field of the given name with a publication timestamp. The\n field with the given name should exist in the provided schema, and be of logical type timestamp\n micros.\n :param includeOnlyColumns: (java.util.function.Predicate) - If not null, filter out any columns tested false in this predicate.\n :param excludeColumns: (java.util.function.Predicate) - If not null, filter out any columns tested true in this predicate.\n :param publishSchema: (boolean) - If true, instead of loading a schema already defined in schema server, define a new Avro\n schema based on the selected columns for this table and publish it to schema server. When publishing,\n if a schema version is provided and the version generated doesn't match, an exception results.\n :param schemaNamespace: (java.lang.String) - When publishSchema is true, the namespace for the generated schema to be restered in\n schema server. When publishSchema is false, null should be passed.\n :param columnProperties: (java.util.Properties) - When publisSchema is true, a Properties object can be provided, specifying\n String properties implying particular Avro type mappings for them. In particular, column X of\n BigDecimal type should specify string properties \"x.precision\" and \"x.scale\".\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A spec corresponding to the schema provided.", "ignoreSpec": "Spec to explicitly ask\n consumeToTable to ignore either key or value.\n\n:return: io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec", - "jsonSpec": "**Incompatible overloads text - text from the first overload:**\n\nA JSON spec from a set of column names\n\n*Overload 1* \n :param includeColumns: (java.lang.String[]) - An array with an entry for each column intended to be included in the JSON output. If\n null, include all columns except those specified in excludeColumns. If includeColumns\n is not null, excludeColumns should be null.\n :param excludeColumns: (java.util.Set) - A set specifying column names to ommit; can only be used when is null.\n In this case all table columns except for the ones in excludeColumns will be included.\n :param columnToFieldMapping: (java.util.Map) - A map from column name to JSON field name to use for that column. Any column\n names implied by earlier arguments not included as a key in the map will be mapped to JSON fields of\n the same name. If null, map all columns to fields of the same name.\n :param nestedObjectDelimiter: (java.lang.String) - if nested JSON fields are desired, the field separator that is used for the\n fieldNames parameter, or null for no nesting. For instance, if a particular column should be mapped to\n JSON field X nested inside field Y, the corresponding field name value for the column\n key in the columnToFieldMapping map can be the string \"X.Y\", in which case the value\n for nestedObjectDelimiter should be {code \".\"}\n :param outputNulls: (boolean) - If false, omit fields with a null value.\n :param timestampFieldName: (java.lang.String) - If not null, include a field of the given name with a publication timestamp.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A JSON spec for the given inputs.\n \n*Overload 2* \n :param includeColumns: (java.lang.String[]) - An array with an entry for each column intended to be included in the JSON output. If\n null, include all columns except those specified in excludeColumns. If includeColumns\n is not null, excludeColumns should be null.\n :param excludeColumns: (java.util.Set) - A set specifying column names to ommit; can only be used when is null.\n In this case all table columns except for the ones in excludeColumns will be included.\n :param columnToFieldMapping: (java.util.Map) - A map from column name to JSON field name to use for that column. Any column\n names implied by earlier arguments not included as a key in the map will be mapped to JSON fields of\n the same name. If null, map all columns to fields of the same name.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A JSON spec for the given inputs.", + "jsonSpec": "**Incompatible overloads text - text from the first overload:**\n\nA JSON spec from a set of column names\n\n*Overload 1* \n :param includeColumns: (java.lang.String[]) - An array with an entry for each column intended to be included in the JSON output. If\n null, include all columns except those specified in excludeColumns. If includeColumns\n is not null, excludeColumns should be null.\n :param excludeColumns: (java.util.function.Predicate) - A set specifying column names to ommit; can only be used when is null.\n In this case all table columns except for the ones in excludeColumns will be included.\n :param columnToFieldMapping: (java.util.Map) - A map from column name to JSON field name to use for that column. Any column\n names implied by earlier arguments not included as a key in the map will be mapped to JSON fields of\n the same name. If null, map all columns to fields of the same name.\n :param nestedObjectDelimiter: (java.lang.String) - if nested JSON fields are desired, the field separator that is used for the\n fieldNames parameter, or null for no nesting. For instance, if a particular column should be mapped to\n JSON field X nested inside field Y, the corresponding field name value for the column\n key in the columnToFieldMapping map can be the string \"X.Y\", in which case the value\n for nestedObjectDelimiter should be {code \".\"}\n :param outputNulls: (boolean) - If false, omit fields with a null value.\n :param timestampFieldName: (java.lang.String) - If not null, include a field of the given name with a publication timestamp.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A JSON spec for the given inputs.\n \n*Overload 2* \n :param includeColumns: (java.lang.String[]) - An array with an entry for each column intended to be included in the JSON output. If\n null, include all columns except those specified in excludeColumns. If includeColumns\n is not null, excludeColumns should be null.\n :param excludeColumns: (java.util.function.Predicate) - A predicate specifying column names to ommit; can only be used when is\n null. In this case all table columns except for the ones in excludeColumns will be included.\n :param columnToFieldMapping: (java.util.Map) - A map from column name to JSON field name to use for that column. Any column\n names implied by earlier arguments not included as a key in the map will be mapped to JSON fields of\n the same name. If null, map all columns to fields of the same name.\n :return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A JSON spec for the given inputs.", "simpleSpec": "A simple spec for sending one column as either key or value in a Kafka message.\n\n:param columnName: (java.lang.String) - The name of the column to include.\n:return: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) A simple spec for the given input." }, "path": "io.deephaven.kafka.KafkaTools.Produce", diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin.json new file mode 100644 index 00000000000..e360b4530e6 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.plugin.Plugin", + "methods": { + "walk": "Note: Java generics information - >\n\n:param visitor: V\n:return: T" + }, + "path": "io.deephaven.plugin.Plugin", + "text": "A plugin is a structured extension point for user-definable behavior.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin/Visitor.json new file mode 100644 index 00000000000..59069de6d26 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/Plugin/Visitor.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.plugin.Plugin$Visitor", + "methods": { + "visit": ":param objectType: io.deephaven.plugin.type.ObjectType\n:return: Plugin.Visitor.T" + }, + "path": "io.deephaven.plugin.Plugin.Visitor", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/Registration.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/Registration.json new file mode 100644 index 00000000000..ff944089c9a --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/Registration.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.plugin.Registration", + "methods": { + "registerInto": "The registration entrypoint.\n\n \n May be called multiple times.\n\n:param callback: (io.deephaven.plugin.Registration.Callback) - the callback." + }, + "path": "io.deephaven.plugin.Registration", + "text": "The registration interface for plugins.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/Registration/Callback.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/Registration/Callback.json new file mode 100644 index 00000000000..2f1b24dd339 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/Registration/Callback.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.plugin.Registration$Callback", + "methods": { + "register": "Registers plugin.\n\n:param plugin: (io.deephaven.plugin.Plugin) - the plugin" + }, + "path": "io.deephaven.plugin.Registration.Callback", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType.json new file mode 100644 index 00000000000..00253ab4a77 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.plugin.type.ObjectType", + "methods": { + "isType": "Returns true if, and only if, the object is compatible with this object type.\n\n:param object: (java.lang.Object) - the object\n:return: (boolean) true if the object is compatible", + "name": "The name of the object type.\n\n:return: (java.lang.String) the name", + "writeTo": "Serializes object into out. Must only be called with a compatible object, see\n isType(Object).\n\n \n Objects that object references may be serialized as ObjectType.Exporter.Reference.\n\n \n Note: the implementation should not hold onto references nor create references outside the calling thread.\n\n:param exporter: (io.deephaven.plugin.type.ObjectType.Exporter) - the exporter\n:param object: (java.lang.Object) - the (compatible) object\n:param out: (java.io.OutputStream) - the output stream" + }, + "path": "io.deephaven.plugin.type.ObjectType", + "text": "An \"object type\" plugin. Useful for serializing custom objects between the server / client.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter.json new file mode 100644 index 00000000000..2606e05f88c --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.plugin.type.ObjectType$Exporter", + "methods": { + "reference": "**Incompatible overloads text - text from the first overload:**\n\nGets the reference for object if it has already been created and forceNew is false,\n otherwise creates a new one. If allowUnknownType is false, and no type can be found, no\n reference will be created.\n\n*Overload 1* \n :param object: (java.lang.Object) - the object\n :param allowUnknownType: (boolean) - if an unknown-typed reference can be created\n :param forceNew: (boolean) - if a new reference should be created\n :return: (java.util.Optional) the reference\n \n*Overload 2* \n :param object: (java.lang.Object) - the object\n :param allowUnknownType: (boolean) - if an unknown-typed reference can be created\n :param forceNew: (boolean) - if a new reference should be created\n :param equals: (java.util.function.BiPredicate) - the equals logic\n :return: (java.util.Optional) the reference" + }, + "path": "io.deephaven.plugin.type.ObjectType.Exporter", + "text": "The interface for creating new references during the ObjectType.writeTo(Exporter, Object, OutputStream).", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter/Reference.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter/Reference.json new file mode 100644 index 00000000000..daf72386af9 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectType/Exporter/Reference.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.plugin.type.ObjectType$Exporter$Reference", + "methods": { + "index": "The index, which is defined by the order in which references are created. May be used in the output\n stream to refer to the reference from the client.\n\n:return: (int) the index", + "type": "The type.\n\n:return: (java.util.Optional) the type, if present" + }, + "path": "io.deephaven.plugin.type.ObjectType.Exporter.Reference", + "text": "A reference.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeBase.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeBase.json new file mode 100644 index 00000000000..bbb50895fca --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeBase.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.plugin.type.ObjectTypeBase", + "methods": { + "walk": "Note: Java generics information - >\n\n:param visitor: V\n:return: T", + "writeCompatibleObjectTo": ":param exporter: io.deephaven.plugin.type.ObjectType.Exporter\n:param object: java.lang.Object\n:param out: java.io.OutputStream", + "writeTo": "Serializes object into out. Must only be called with a compatible object, see\n ObjectType.isType(Object).\n\n \n Objects that object references may be serialized as ObjectType.Exporter.Reference.\n\n \n Note: the implementation should not hold onto references nor create references outside the calling thread.\n\n:param exporter: (io.deephaven.plugin.type.ObjectType.Exporter) - the exporter\n:param object: (java.lang.Object) - the (compatible) object\n:param out: (java.io.OutputStream) - the output stream" + }, + "path": "io.deephaven.plugin.type.ObjectTypeBase", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeClassBase.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeClassBase.json new file mode 100644 index 00000000000..b7754b5afb5 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeClassBase.json @@ -0,0 +1,14 @@ +{ + "className": "io.deephaven.plugin.type.ObjectTypeClassBase", + "methods": { + "clazz": ":return: java.lang.Class", + "isType": "Returns true if, and only if, the object is compatible with this object type.\n\n:param object: (java.lang.Object) - the object\n:return: (boolean) true if the object is compatible", + "name": "The name of the object type.\n\n:return: (java.lang.String) the name", + "toString": ":return: java.lang.String", + "writeCompatibleObjectTo": ":param exporter: io.deephaven.plugin.type.ObjectType.Exporter\n:param object: java.lang.Object\n:param out: java.io.OutputStream", + "writeToImpl": ":param exporter: io.deephaven.plugin.type.ObjectType.Exporter\n:param object: ObjectTypeClassBase.T\n:param out: java.io.OutputStream" + }, + "path": "io.deephaven.plugin.type.ObjectTypeClassBase", + "text": "An implementation that uses strict Class equality for the isType(Object) check.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup.json new file mode 100644 index 00000000000..cb64e7343ca --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.plugin.type.ObjectTypeLookup", + "methods": { + "findObjectType": "Find the ObjectType compatible with object. That is, ObjectType.isType(Object) will be\n true for object.\n\n:param object: (java.lang.Object) - the object\n:return: (java.util.Optional) the object type, if found" + }, + "path": "io.deephaven.plugin.type.ObjectTypeLookup", + "text": "The object type lookup.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup/NoOp.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup/NoOp.json new file mode 100644 index 00000000000..3ac51c31be4 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeLookup/NoOp.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.plugin.type.ObjectTypeLookup$NoOp", + "methods": { + "findObjectType": "Find the ObjectType compatible with object. That is, ObjectType.isType(Object) will be\n true for object.\n\n:param object: (java.lang.Object) - the object\n:return: (java.util.Optional) the object type, if found", + "valueOf": "Returns the enum constant of this type with the specified name.\nThe string must match exactly an identifier used to declare an\nenum constant in this type. (Extraneous whitespace characters are \nnot permitted.)\n\n:param name: (java.lang.String) - the name of the enum constant to be returned.\n:return: (io.deephaven.plugin.type.ObjectTypeLookup.NoOp) the enum constant with the specified name", + "values": "Returns an array containing the constants of this enum type, in\nthe order they are declared.\n\n:return: (io.deephaven.plugin.type.ObjectTypeLookup.NoOp[]) an array containing the constants of this enum type, in the order they are declared" + }, + "path": "io.deephaven.plugin.type.ObjectTypeLookup.NoOp", + "typeName": "enum" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeRegistration.json b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeRegistration.json new file mode 100644 index 00000000000..e6de7ca2c60 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/plugin/type/ObjectTypeRegistration.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.plugin.type.ObjectTypeRegistration", + "methods": { + "register": "Register objectType.\n\n:param objectType: (io.deephaven.plugin.type.ObjectType) - the object type" + }, + "path": "io.deephaven.plugin.type.ObjectTypeRegistration", + "text": "The ObjectType specific registration.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/DeephavenChannel.json b/Integrations/python/deephaven/doc/io/deephaven/proto/DeephavenChannel.json index 2fcf9b39cb2..9c2029833c5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/DeephavenChannel.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/DeephavenChannel.json @@ -11,6 +11,9 @@ "inputTable": ":return: io.deephaven.proto.backplane.grpc.InputTableServiceGrpc.InputTableServiceStub", "inputTableBlocking": ":return: io.deephaven.proto.backplane.grpc.InputTableServiceGrpc.InputTableServiceBlockingStub", "inputTableFuture": ":return: io.deephaven.proto.backplane.grpc.InputTableServiceGrpc.InputTableServiceFutureStub", + "object": ":return: io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceStub", + "objectBlocking": ":return: io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceBlockingStub", + "objectFuture": ":return: io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceFutureStub", "session": ":return: io.deephaven.proto.backplane.grpc.SessionServiceGrpc.SessionServiceStub", "sessionBlocking": ":return: io.deephaven.proto.backplane.grpc.SessionServiceGrpc.SessionServiceBlockingStub", "sessionFuture": ":return: io.deephaven.proto.backplane.grpc.SessionServiceGrpc.SessionServiceFutureStub", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo.json deleted file mode 100644 index 82adf83945e..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo.json +++ /dev/null @@ -1,26 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.CustomInfo", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.CustomInfo", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.CustomInfo", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getType": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (java.lang.String) The type.", - "getTypeBytes": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.CustomInfo\n :return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.CustomInfo\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.grpc.CustomInfo", - "text": "CustomInfo can be used to provide information about an exposed field in an unstructured way.\n Alternatively, one can extend FieldType and re-build with custom types of their own structure.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.CustomInfo", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo/Builder.json deleted file mode 100644 index cf288904cde..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfo/Builder.json +++ /dev/null @@ -1,29 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.CustomInfo$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "build": ":return: io.deephaven.proto.backplane.grpc.CustomInfo", - "buildPartial": ":return: io.deephaven.proto.backplane.grpc.CustomInfo", - "clear": ":return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "clearType": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.CustomInfo.Builder) This builder for chaining.", - "clone": ":return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.CustomInfo", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getType": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (java.lang.String) The type.", - "getTypeBytes": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", - "isInitialized": ":return: boolean", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.CustomInfo\n :return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "setType": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:param value: (java.lang.String) - The type to set.\n:return: (io.deephaven.proto.backplane.grpc.CustomInfo.Builder) This builder for chaining.", - "setTypeBytes": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for type to set.\n:return: (io.deephaven.proto.backplane.grpc.CustomInfo.Builder) This builder for chaining.", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder" - }, - "path": "io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "text": "CustomInfo can be used to provide information about an exposed field in an unstructured way.\n Alternatively, one can extend FieldType and re-build with custom types of their own structure.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.CustomInfo", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfoOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfoOrBuilder.json deleted file mode 100644 index fa8318e5a26..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/CustomInfoOrBuilder.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.CustomInfoOrBuilder", - "methods": { - "getType": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (java.lang.String) The type.", - "getTypeBytes": "application-specific type to identify the purpose of this custom field reference\n \nstring type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type." - }, - "path": "io.deephaven.proto.backplane.grpc.CustomInfoOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest.json new file mode 100644 index 00000000000..8d0ee91112a --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest.json @@ -0,0 +1,27 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "methods": { + "equals": ":param obj: java.lang.Object\n:return: boolean", + "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", + "getParserForType": ":return: com.google.protobuf.Parser", + "getSerializedSize": ":return: int", + "getSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket) The sourceId.", + "getSourceIdOrBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", + "hasSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: (boolean) Whether the sourceId field is set.", + "hashCode": ":return: int", + "isInitialized": ":return: boolean", + "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "parser": ":return: com.google.protobuf.Parser", + "toBuilder": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "writeTo": ":param output: com.google.protobuf.CodedOutputStream" + }, + "path": "io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "text": "Protobuf type io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest/Builder.json new file mode 100644 index 00000000000..b73d318aefb --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequest/Builder.json @@ -0,0 +1,31 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.FetchObjectRequest$Builder", + "methods": { + "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "build": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "buildPartial": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "clear": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "clearSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "clone": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", + "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", + "getSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket) The sourceId.", + "getSourceIdBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "getSourceIdOrBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "hasSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: (boolean) Whether the sourceId field is set.", + "isInitialized": ":return: boolean", + "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "mergeSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:param value: io.deephaven.proto.backplane.grpc.TypedTicket\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "setSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n :return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder" + }, + "path": "io.deephaven.proto.backplane.grpc.FetchObjectRequest.Builder", + "text": "Protobuf type io.deephaven.proto.backplane.grpc.FetchObjectRequest", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequestOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequestOrBuilder.json new file mode 100644 index 00000000000..cedbb54ad56 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectRequestOrBuilder.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.FetchObjectRequestOrBuilder", + "methods": { + "getSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket) The sourceId.", + "getSourceIdOrBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "hasSourceId": ".io.deephaven.proto.backplane.grpc.TypedTicket source_id = 1;\n\n:return: (boolean) Whether the sourceId field is set." + }, + "path": "io.deephaven.proto.backplane.grpc.FetchObjectRequestOrBuilder", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse.json new file mode 100644 index 00000000000..95688d8e017 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse.json @@ -0,0 +1,32 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "methods": { + "equals": ":param obj: java.lang.Object\n:return: boolean", + "getData": "bytes data = 2;\n\n:return: (com.google.protobuf.ByteString) The data.", + "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", + "getParserForType": ":return: com.google.protobuf.Parser", + "getSerializedSize": ":return: int", + "getType": "string type = 1;\n\n:return: (java.lang.String) The type.", + "getTypeBytes": "string type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", + "getTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicket", + "getTypedExportIdCount": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: int", + "getTypedExportIdList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List", + "getTypedExportIdOrBuilder": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "getTypedExportIdOrBuilderList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List", + "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", + "hashCode": ":return: int", + "isInitialized": ":return: boolean", + "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "parser": ":return: com.google.protobuf.Parser", + "toBuilder": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "writeTo": ":param output: com.google.protobuf.CodedOutputStream" + }, + "path": "io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "text": "Protobuf type io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse/Builder.json new file mode 100644 index 00000000000..9a6b6a1e82f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponse/Builder.json @@ -0,0 +1,45 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.FetchObjectResponse$Builder", + "methods": { + "addAllTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param values: java.lang.Iterable\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "addTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 2* \n :param index: int\n :param value: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 3* \n :param builderForValue: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 4* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "addTypedExportIdBuilder": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n \n*Overload 2* \n :param index: int\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "build": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "buildPartial": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "clear": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "clearData": "bytes data = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder) This builder for chaining.", + "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "clearType": "string type = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder) This builder for chaining.", + "clearTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "clone": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "getData": "bytes data = 2;\n\n:return: (com.google.protobuf.ByteString) The data.", + "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", + "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", + "getType": "string type = 1;\n\n:return: (java.lang.String) The type.", + "getTypeBytes": "string type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", + "getTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicket", + "getTypedExportIdBuilder": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "getTypedExportIdBuilderList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List", + "getTypedExportIdCount": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: int", + "getTypedExportIdList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List", + "getTypedExportIdOrBuilder": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "getTypedExportIdOrBuilderList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List", + "isInitialized": ":return: boolean", + "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.FetchObjectResponse\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "removeTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "setData": "bytes data = 2;\n\n:param value: (com.google.protobuf.ByteString) - The data to set.\n:return: (io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder) This builder for chaining.", + "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "setType": "string type = 1;\n\n:param value: (java.lang.String) - The type to set.\n:return: (io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder) This builder for chaining.", + "setTypeBytes": "string type = 1;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for type to set.\n:return: (io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder) This builder for chaining.", + "setTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n*Overload 1* \n :param index: int\n :param value: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder\n \n*Overload 2* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n :return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder" + }, + "path": "io.deephaven.proto.backplane.grpc.FetchObjectResponse.Builder", + "text": "Protobuf type io.deephaven.proto.backplane.grpc.FetchObjectResponse", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponseOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponseOrBuilder.json new file mode 100644 index 00000000000..d40419714f3 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FetchObjectResponseOrBuilder.json @@ -0,0 +1,15 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.FetchObjectResponseOrBuilder", + "methods": { + "getData": "bytes data = 2;\n\n:return: (com.google.protobuf.ByteString) The data.", + "getType": "string type = 1;\n\n:return: (java.lang.String) The type.", + "getTypeBytes": "string type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", + "getTypedExportId": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicket", + "getTypedExportIdCount": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: int", + "getTypedExportIdList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List", + "getTypedExportIdOrBuilder": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "getTypedExportIdOrBuilderList": "repeated .io.deephaven.proto.backplane.grpc.TypedTicket typed_export_id = 3;\n\n:return: java.util.List" + }, + "path": "io.deephaven.proto.backplane.grpc.FetchObjectResponseOrBuilder", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo.json index 177be82b84f..bf942a99e91 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo.json @@ -2,26 +2,23 @@ "className": "io.deephaven.proto.backplane.grpc.FieldInfo", "methods": { "equals": ":param obj: java.lang.Object\n:return: boolean", - "getApplicationId": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (java.lang.String) The applicationId.", - "getApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationId.", - "getApplicationName": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (java.lang.String) The applicationName.", - "getApplicationNameBytes": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationName.", + "getApplicationId": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (java.lang.String) The applicationId.", + "getApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationId.", + "getApplicationName": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (java.lang.String) The applicationName.", + "getApplicationNameBytes": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationName.", "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.FieldInfo", "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FieldInfo", "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getFieldDescription": "string field_description = 4;\n\n:return: (java.lang.String) The fieldDescription.", - "getFieldDescriptionBytes": "string field_description = 4;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldDescription.", + "getFieldDescription": "string field_description = 3;\n\n:return: (java.lang.String) The fieldDescription.", + "getFieldDescriptionBytes": "string field_description = 3;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldDescription.", "getFieldName": "string field_name = 2;\n\n:return: (java.lang.String) The fieldName.", "getFieldNameBytes": "string field_name = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldName.", - "getFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.FieldType) The fieldType.", - "getFieldTypeOrBuilder": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldTypeOrBuilder", "getParserForType": ":return: com.google.protobuf.Parser", "getSerializedSize": ":return: int", - "getTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The ticket.", - "getTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", + "getTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket) The typedTicket.", + "getTypedTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hasFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: (boolean) Whether the fieldType field is set.", - "hasTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (boolean) Whether the ticket field is set.", + "hasTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: (boolean) Whether the typedTicket field is set.", "hashCode": ":return: int", "isInitialized": ":return: boolean", "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.FieldInfo\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/Builder.json index f857c81d5d7..ad04f57299d 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/Builder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/Builder.json @@ -5,51 +5,44 @@ "build": ":return: io.deephaven.proto.backplane.grpc.FieldInfo", "buildPartial": ":return: io.deephaven.proto.backplane.grpc.FieldInfo", "clear": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "clearApplicationId": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "clearApplicationName": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "clearApplicationId": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "clearApplicationName": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "clearFieldDescription": "string field_description = 4;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "clearFieldDescription": "string field_description = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", "clearFieldName": "string field_name = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "clearFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "clearTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", + "clearTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", "clone": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "getApplicationId": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (java.lang.String) The applicationId.", - "getApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationId.", - "getApplicationName": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (java.lang.String) The applicationName.", - "getApplicationNameBytes": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationName.", + "getApplicationId": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (java.lang.String) The applicationId.", + "getApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationId.", + "getApplicationName": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (java.lang.String) The applicationName.", + "getApplicationNameBytes": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationName.", "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FieldInfo", "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getFieldDescription": "string field_description = 4;\n\n:return: (java.lang.String) The fieldDescription.", - "getFieldDescriptionBytes": "string field_description = 4;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldDescription.", + "getFieldDescription": "string field_description = 3;\n\n:return: (java.lang.String) The fieldDescription.", + "getFieldDescriptionBytes": "string field_description = 3;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldDescription.", "getFieldName": "string field_name = 2;\n\n:return: (java.lang.String) The fieldName.", "getFieldNameBytes": "string field_name = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldName.", - "getFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.FieldType) The fieldType.", - "getFieldTypeBuilder": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "getFieldTypeOrBuilder": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldTypeOrBuilder", - "getTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The ticket.", - "getTicketBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.Ticket.Builder", - "getTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "hasFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: (boolean) Whether the fieldType field is set.", - "hasTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (boolean) Whether the ticket field is set.", + "getTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket) The typedTicket.", + "getTypedTicketBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "getTypedTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "hasTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: (boolean) Whether the typedTicket field is set.", "isInitialized": ":return: boolean", - "mergeFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:param value: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.FieldInfo\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "mergeTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:param value: io.deephaven.proto.backplane.grpc.Ticket\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", + "mergeTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:param value: io.deephaven.proto.backplane.grpc.TypedTicket\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "setApplicationId": "computer-friendly identification\n \nstring application_id = 6;\n\n:param value: (java.lang.String) - The applicationId to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "setApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 6;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for applicationId to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "setApplicationName": "display-friendly identification\n \nstring application_name = 5;\n\n:param value: (java.lang.String) - The applicationName to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "setApplicationNameBytes": "display-friendly identification\n \nstring application_name = 5;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for applicationName to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "setApplicationId": "computer-friendly identification\n \nstring application_id = 5;\n\n:param value: (java.lang.String) - The applicationId to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "setApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 5;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for applicationId to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "setApplicationName": "display-friendly identification\n \nstring application_name = 4;\n\n:param value: (java.lang.String) - The applicationName to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "setApplicationNameBytes": "display-friendly identification\n \nstring application_name = 4;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for applicationName to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "setFieldDescription": "string field_description = 4;\n\n:param value: (java.lang.String) - The fieldDescription to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "setFieldDescriptionBytes": "string field_description = 4;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for fieldDescription to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "setFieldDescription": "string field_description = 3;\n\n:param value: (java.lang.String) - The fieldDescription to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", + "setFieldDescriptionBytes": "string field_description = 3;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for fieldDescription to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", "setFieldName": "string field_name = 2;\n\n:param value: (java.lang.String) - The fieldName to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", "setFieldNameBytes": "string field_name = 2;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for fieldName to set.\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.Builder) This builder for chaining.", - "setFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", - "setTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.Ticket\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.Ticket.Builder\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", + "setTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder", "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.Builder" }, "path": "io.deephaven.proto.backplane.grpc.FieldInfo.Builder", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType.json deleted file mode 100644 index b69b8d480a9..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType.json +++ /dev/null @@ -1,34 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FieldInfo$FieldType", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.CustomInfo) The custom.", - "getCustomOrBuilder": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: io.deephaven.proto.backplane.grpc.CustomInfoOrBuilder", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getFieldCase": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase", - "getFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: (io.deephaven.proto.backplane.grpc.FigureInfo) The figure.", - "getFigureOrBuilder": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: io.deephaven.proto.backplane.grpc.FigureInfoOrBuilder", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.TableInfo) The table.", - "getTableOrBuilder": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: io.deephaven.proto.backplane.grpc.TableInfoOrBuilder", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hasCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: (boolean) Whether the custom field is set.", - "hasFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: (boolean) Whether the figure field is set.", - "hasTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: (boolean) Whether the table field is set.", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "text": "Protobuf type io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/Builder.json deleted file mode 100644 index b1fef49e208..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/Builder.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FieldInfo$FieldType$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "build": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "buildPartial": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "clear": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "clearCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "clearField": "*Overload 1* \n :param field: com.google.protobuf.Descriptors.FieldDescriptor\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 2* \n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "clearFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "clearTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "clone": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "getCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.CustomInfo) The custom.", - "getCustomBuilder": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: io.deephaven.proto.backplane.grpc.CustomInfo.Builder", - "getCustomOrBuilder": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: io.deephaven.proto.backplane.grpc.CustomInfoOrBuilder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getFieldCase": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase", - "getFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: (io.deephaven.proto.backplane.grpc.FigureInfo) The figure.", - "getFigureBuilder": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "getFigureOrBuilder": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: io.deephaven.proto.backplane.grpc.FigureInfoOrBuilder", - "getTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.TableInfo) The table.", - "getTableBuilder": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "getTableOrBuilder": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: io.deephaven.proto.backplane.grpc.TableInfoOrBuilder", - "hasCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: (boolean) Whether the custom field is set.", - "hasFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: (boolean) Whether the figure field is set.", - "hasTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: (boolean) Whether the table field is set.", - "isInitialized": ":return: boolean", - "mergeCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:param value: io.deephaven.proto.backplane.grpc.CustomInfo\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "mergeFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:param value: io.deephaven.proto.backplane.grpc.FigureInfo\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "mergeTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:param value: io.deephaven.proto.backplane.grpc.TableInfo\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "setCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.CustomInfo\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.CustomInfo.Builder\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "setFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.FigureInfo\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.FigureInfo.Builder\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "setTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.TableInfo\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.TableInfo.Builder\n :return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder" - }, - "path": "io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.Builder", - "text": "Protobuf type io.deephaven.proto.backplane.grpc.FieldInfo.FieldType", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/FieldCase.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/FieldCase.json deleted file mode 100644 index d24a3df3b95..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldType/FieldCase.json +++ /dev/null @@ -1,11 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FieldInfo$FieldType$FieldCase", - "methods": { - "forNumber": ":param value: int\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase", - "getNumber": ":return: int", - "valueOf": "Returns the enum constant of this type with the specified name.\nThe string must match exactly an identifier used to declare an\nenum constant in this type. (Extraneous whitespace characters are \nnot permitted.)\n\n*Overload 1* \n :param name: (java.lang.String) - the name of the enum constant to be returned.\n :return: (io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase) the enum constant with the specified name\n \n*Overload 2* \n :param value: (int) - the name of the enum constant to be returned.\n :return: (io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase) the enum constant with the specified name", - "values": "Returns an array containing the constants of this enum type, in\nthe order they are declared.\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase[]) an array containing the constants of this enum type, in the order they are declared" - }, - "path": "io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase", - "typeName": "enum" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldTypeOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldTypeOrBuilder.json deleted file mode 100644 index 5299643c7a7..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfo/FieldTypeOrBuilder.json +++ /dev/null @@ -1,17 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FieldInfo$FieldTypeOrBuilder", - "methods": { - "getCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.CustomInfo) The custom.", - "getCustomOrBuilder": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: io.deephaven.proto.backplane.grpc.CustomInfoOrBuilder", - "getFieldCase": ":return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldType.FieldCase", - "getFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: (io.deephaven.proto.backplane.grpc.FigureInfo) The figure.", - "getFigureOrBuilder": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: io.deephaven.proto.backplane.grpc.FigureInfoOrBuilder", - "getTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.TableInfo) The table.", - "getTableOrBuilder": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: io.deephaven.proto.backplane.grpc.TableInfoOrBuilder", - "hasCustom": ".io.deephaven.proto.backplane.grpc.CustomInfo custom = 1;\n\n:return: (boolean) Whether the custom field is set.", - "hasFigure": "reserved = 3; // for TreeTable\n reserved = 4; // for TableMap\n \n.io.deephaven.proto.backplane.grpc.FigureInfo figure = 5;\n\n:return: (boolean) Whether the figure field is set.", - "hasTable": ".io.deephaven.proto.backplane.grpc.TableInfo table = 2;\n\n:return: (boolean) Whether the table field is set." - }, - "path": "io.deephaven.proto.backplane.grpc.FieldInfo.FieldTypeOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfoOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfoOrBuilder.json index 4af8e33f982..92b1269d50c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfoOrBuilder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FieldInfoOrBuilder.json @@ -1,20 +1,17 @@ { "className": "io.deephaven.proto.backplane.grpc.FieldInfoOrBuilder", "methods": { - "getApplicationId": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (java.lang.String) The applicationId.", - "getApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 6;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationId.", - "getApplicationName": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (java.lang.String) The applicationName.", - "getApplicationNameBytes": "display-friendly identification\n \nstring application_name = 5;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationName.", - "getFieldDescription": "string field_description = 4;\n\n:return: (java.lang.String) The fieldDescription.", - "getFieldDescriptionBytes": "string field_description = 4;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldDescription.", + "getApplicationId": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (java.lang.String) The applicationId.", + "getApplicationIdBytes": "computer-friendly identification\n \nstring application_id = 5;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationId.", + "getApplicationName": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (java.lang.String) The applicationName.", + "getApplicationNameBytes": "display-friendly identification\n \nstring application_name = 4;\n\n:return: (com.google.protobuf.ByteString) The bytes for applicationName.", + "getFieldDescription": "string field_description = 3;\n\n:return: (java.lang.String) The fieldDescription.", + "getFieldDescriptionBytes": "string field_description = 3;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldDescription.", "getFieldName": "string field_name = 2;\n\n:return: (java.lang.String) The fieldName.", "getFieldNameBytes": "string field_name = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for fieldName.", - "getFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldInfo.FieldType) The fieldType.", - "getFieldTypeOrBuilder": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: io.deephaven.proto.backplane.grpc.FieldInfo.FieldTypeOrBuilder", - "getTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The ticket.", - "getTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "hasFieldType": ".io.deephaven.proto.backplane.grpc.FieldInfo.FieldType field_type = 3;\n\n:return: (boolean) Whether the fieldType field is set.", - "hasTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (boolean) Whether the ticket field is set." + "getTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket) The typedTicket.", + "getTypedTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "hasTypedTicket": ".io.deephaven.proto.backplane.grpc.TypedTicket typed_ticket = 1;\n\n:return: (boolean) Whether the typedTicket field is set." }, "path": "io.deephaven.proto.backplane.grpc.FieldInfoOrBuilder", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo.json deleted file mode 100644 index b6a390e6667..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo.json +++ /dev/null @@ -1,24 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FigureInfo", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.FigureInfo", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FigureInfo", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.FigureInfo\n :return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.FigureInfo\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.grpc.FigureInfo", - "text": "Lightly describes a Figure.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.FigureInfo", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo/Builder.json deleted file mode 100644 index 73c5b4d9814..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfo/Builder.json +++ /dev/null @@ -1,24 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FigureInfo$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "build": ":return: io.deephaven.proto.backplane.grpc.FigureInfo", - "buildPartial": ":return: io.deephaven.proto.backplane.grpc.FigureInfo", - "clear": ":return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "clone": ":return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.FigureInfo", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "isInitialized": ":return: boolean", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.FigureInfo\n :return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.FigureInfo.Builder" - }, - "path": "io.deephaven.proto.backplane.grpc.FigureInfo.Builder", - "text": "Lightly describes a Figure.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.FigureInfo", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfoOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfoOrBuilder.json deleted file mode 100644 index 213347e1c89..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/FigureInfoOrBuilder.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.FigureInfoOrBuilder", - "methods": {}, - "path": "io.deephaven.proto.backplane.grpc.FigureInfoOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/Object.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/Object.json new file mode 100644 index 00000000000..437c5032087 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/Object.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.Object", + "methods": { + "getDescriptor": ":return: com.google.protobuf.Descriptors.FileDescriptor", + "registerAllExtensions": "*Overload 1* \n :param registry: com.google.protobuf.ExtensionRegistryLite\n \n*Overload 2* \n :param registry: com.google.protobuf.ExtensionRegistry" + }, + "path": "io.deephaven.proto.backplane.grpc.Object", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc.json new file mode 100644 index 00000000000..f747008ae94 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc.json @@ -0,0 +1,12 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc", + "methods": { + "getFetchObjectMethod": ":return: io.grpc.MethodDescriptor", + "getServiceDescriptor": ":return: io.grpc.ServiceDescriptor", + "newBlockingStub": "Creates a new blocking-style stub that supports unary and streaming output calls on the service\n\n:param channel: io.grpc.Channel\n:return: io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceBlockingStub", + "newFutureStub": "Creates a new ListenableFuture-style stub that supports unary calls on the service\n\n:param channel: io.grpc.Channel\n:return: io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceFutureStub", + "newStub": "Creates a new async stub that supports all call types for the service\n\n:param channel: io.grpc.Channel\n:return: io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceStub" + }, + "path": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceBlockingStub.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceBlockingStub.json new file mode 100644 index 00000000000..81e763ad417 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceBlockingStub.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc$ObjectServiceBlockingStub", + "methods": { + "fetchObject": ":param request: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n:return: io.deephaven.proto.backplane.grpc.FetchObjectResponse" + }, + "path": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceBlockingStub", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceFutureStub.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceFutureStub.json new file mode 100644 index 00000000000..a6e5b19789f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceFutureStub.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc$ObjectServiceFutureStub", + "methods": { + "fetchObject": ":param request: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n:return: com.google.common.util.concurrent.ListenableFuture" + }, + "path": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceFutureStub", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceImplBase.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceImplBase.json new file mode 100644 index 00000000000..9d904812587 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceImplBase.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc$ObjectServiceImplBase", + "methods": { + "bindService": ":return: io.grpc.ServerServiceDefinition", + "fetchObject": ":param request: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n:param responseObserver: io.grpc.stub.StreamObserver" + }, + "path": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceImplBase", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceStub.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceStub.json new file mode 100644 index 00000000000..d34451ecb39 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/ObjectServiceGrpc/ObjectServiceStub.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc$ObjectServiceStub", + "methods": { + "fetchObject": ":param request: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n:param responseObserver: io.grpc.stub.StreamObserver" + }, + "path": "io.deephaven.proto.backplane.grpc.ObjectServiceGrpc.ObjectServiceStub", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField.json deleted file mode 100644 index b4c4a1f9e36..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField.json +++ /dev/null @@ -1,24 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.RemovedField", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.RemovedField", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.RemovedField", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.RemovedField.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.RemovedField\n :return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.RemovedField\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.grpc.RemovedField", - "text": "RemovedField represents that the provided ticket is no longer available.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.RemovedField", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField/Builder.json deleted file mode 100644 index 4148182a8ef..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedField/Builder.json +++ /dev/null @@ -1,24 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.RemovedField$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "build": ":return: io.deephaven.proto.backplane.grpc.RemovedField", - "buildPartial": ":return: io.deephaven.proto.backplane.grpc.RemovedField", - "clear": ":return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "clone": ":return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.RemovedField", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "isInitialized": ":return: boolean", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.RemovedField.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.RemovedField\n :return: io.deephaven.proto.backplane.grpc.RemovedField.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.RemovedField.Builder" - }, - "path": "io.deephaven.proto.backplane.grpc.RemovedField.Builder", - "text": "RemovedField represents that the provided ticket is no longer available.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.RemovedField", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedFieldOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedFieldOrBuilder.json deleted file mode 100644 index 2a2cb23def0..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/RemovedFieldOrBuilder.json +++ /dev/null @@ -1,6 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.RemovedFieldOrBuilder", - "methods": {}, - "path": "io.deephaven.proto.backplane.grpc.RemovedFieldOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo.json deleted file mode 100644 index e98f4b550fc..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo.json +++ /dev/null @@ -1,27 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.TableInfo", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.TableInfo", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.TableInfo", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getIsStatic": "Whether or not this table might change.\n \nbool is_static = 2;\n\n:return: (boolean) The isStatic.", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSchemaHeader": "Schema as described in Arrow Message.fbs::Message.\n \nbytes schema_header = 1;\n\n:return: (com.google.protobuf.ByteString) The schemaHeader.", - "getSerializedSize": ":return: int", - "getSize": "The current number of rows for this table.\n \nsint64 size = 3 [jstype = JS_STRING];\n\n:return: (long) The size.", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.TableInfo.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.TableInfo\n :return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.TableInfo\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.grpc.TableInfo", - "text": "Lightly describe a Table.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.TableInfo", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo/Builder.json deleted file mode 100644 index d2d46017a80..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfo/Builder.json +++ /dev/null @@ -1,33 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.TableInfo$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "build": ":return: io.deephaven.proto.backplane.grpc.TableInfo", - "buildPartial": ":return: io.deephaven.proto.backplane.grpc.TableInfo", - "clear": ":return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "clearIsStatic": "Whether or not this table might change.\n \nbool is_static = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.TableInfo.Builder) This builder for chaining.", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "clearSchemaHeader": "Schema as described in Arrow Message.fbs::Message.\n \nbytes schema_header = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.TableInfo.Builder) This builder for chaining.", - "clearSize": "The current number of rows for this table.\n \nsint64 size = 3 [jstype = JS_STRING];\n\n:return: (io.deephaven.proto.backplane.grpc.TableInfo.Builder) This builder for chaining.", - "clone": ":return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.TableInfo", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getIsStatic": "Whether or not this table might change.\n \nbool is_static = 2;\n\n:return: (boolean) The isStatic.", - "getSchemaHeader": "Schema as described in Arrow Message.fbs::Message.\n \nbytes schema_header = 1;\n\n:return: (com.google.protobuf.ByteString) The schemaHeader.", - "getSize": "The current number of rows for this table.\n \nsint64 size = 3 [jstype = JS_STRING];\n\n:return: (long) The size.", - "isInitialized": ":return: boolean", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.TableInfo.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.TableInfo\n :return: io.deephaven.proto.backplane.grpc.TableInfo.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "setIsStatic": "Whether or not this table might change.\n \nbool is_static = 2;\n\n:param value: (boolean) - The isStatic to set.\n:return: (io.deephaven.proto.backplane.grpc.TableInfo.Builder) This builder for chaining.", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "setSchemaHeader": "Schema as described in Arrow Message.fbs::Message.\n \nbytes schema_header = 1;\n\n:param value: (com.google.protobuf.ByteString) - The schemaHeader to set.\n:return: (io.deephaven.proto.backplane.grpc.TableInfo.Builder) This builder for chaining.", - "setSize": "The current number of rows for this table.\n \nsint64 size = 3 [jstype = JS_STRING];\n\n:param value: (long) - The size to set.\n:return: (io.deephaven.proto.backplane.grpc.TableInfo.Builder) This builder for chaining.", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.TableInfo.Builder" - }, - "path": "io.deephaven.proto.backplane.grpc.TableInfo.Builder", - "text": "Lightly describe a Table.\n \n\n Protobuf type io.deephaven.proto.backplane.grpc.TableInfo", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfoOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfoOrBuilder.json deleted file mode 100644 index 888513b05f3..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TableInfoOrBuilder.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.grpc.TableInfoOrBuilder", - "methods": { - "getIsStatic": "Whether or not this table might change.\n \nbool is_static = 2;\n\n:return: (boolean) The isStatic.", - "getSchemaHeader": "Schema as described in Arrow Message.fbs::Message.\n \nbytes schema_header = 1;\n\n:return: (com.google.protobuf.ByteString) The schemaHeader.", - "getSize": "The current number of rows for this table.\n \nsint64 size = 3 [jstype = JS_STRING];\n\n:return: (long) The size." - }, - "path": "io.deephaven.proto.backplane.grpc.TableInfoOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket.json new file mode 100644 index 00000000000..be3f03bf803 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket.json @@ -0,0 +1,29 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.TypedTicket", + "methods": { + "equals": ":param obj: java.lang.Object\n:return: boolean", + "getDefaultInstance": ":return: io.deephaven.proto.backplane.grpc.TypedTicket", + "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.TypedTicket", + "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", + "getParserForType": ":return: com.google.protobuf.Parser", + "getSerializedSize": ":return: int", + "getTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The ticket.", + "getTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", + "getType": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (java.lang.String) The type.", + "getTypeBytes": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", + "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", + "hasTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (boolean) Whether the ticket field is set.", + "hashCode": ":return: int", + "isInitialized": ":return: boolean", + "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "newBuilderForType": ":return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket", + "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.grpc.TypedTicket\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket", + "parser": ":return: com.google.protobuf.Parser", + "toBuilder": ":return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "writeTo": ":param output: com.google.protobuf.CodedOutputStream" + }, + "path": "io.deephaven.proto.backplane.grpc.TypedTicket", + "text": "Protobuf type io.deephaven.proto.backplane.grpc.TypedTicket", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket/Builder.json new file mode 100644 index 00000000000..7d7ddf1300f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicket/Builder.json @@ -0,0 +1,36 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.TypedTicket$Builder", + "methods": { + "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "build": ":return: io.deephaven.proto.backplane.grpc.TypedTicket", + "buildPartial": ":return: io.deephaven.proto.backplane.grpc.TypedTicket", + "clear": ":return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "clearTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "clearType": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket.Builder) This builder for chaining.", + "clone": ":return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.grpc.TypedTicket", + "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", + "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", + "getTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The ticket.", + "getTicketBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.Ticket.Builder", + "getTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", + "getType": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (java.lang.String) The type.", + "getTypeBytes": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", + "hasTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (boolean) Whether the ticket field is set.", + "isInitialized": ":return: boolean", + "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.grpc.TypedTicket\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "mergeTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:param value: io.deephaven.proto.backplane.grpc.Ticket\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "setTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.Ticket\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.Ticket.Builder\n :return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "setType": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:param value: (java.lang.String) - The type to set.\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket.Builder) This builder for chaining.", + "setTypeBytes": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for type to set.\n:return: (io.deephaven.proto.backplane.grpc.TypedTicket.Builder) This builder for chaining.", + "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.grpc.TypedTicket.Builder" + }, + "path": "io.deephaven.proto.backplane.grpc.TypedTicket.Builder", + "text": "Protobuf type io.deephaven.proto.backplane.grpc.TypedTicket", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicketOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicketOrBuilder.json new file mode 100644 index 00000000000..5678d732ae5 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/grpc/TypedTicketOrBuilder.json @@ -0,0 +1,12 @@ +{ + "className": "io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "methods": { + "getTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The ticket.", + "getTicketOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", + "getType": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (java.lang.String) The type.", + "getTypeBytes": "The type. An empty string means that it is not known, not that the server chose to not set it.\n \nstring type = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", + "hasTicket": ".io.deephaven.proto.backplane.grpc.Ticket ticket = 1;\n\n:return: (boolean) Whether the ticket field is set." + }, + "path": "io.deephaven.proto.backplane.grpc.TypedTicketOrBuilder", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc.json index cabe7bf4f0e..a21ab31b069 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc.json @@ -5,7 +5,6 @@ "getBindTableToVariableMethod": ":return: io.grpc.MethodDescriptor", "getCancelCommandMethod": ":return: io.grpc.MethodDescriptor", "getExecuteCommandMethod": ":return: io.grpc.MethodDescriptor", - "getFetchFigureMethod": ":return: io.grpc.MethodDescriptor", "getGetConsoleTypesMethod": ":return: io.grpc.MethodDescriptor", "getNextAutoCompleteStreamMethod": ":return: io.grpc.MethodDescriptor", "getOpenAutoCompleteStreamMethod": ":return: io.grpc.MethodDescriptor", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceBlockingStub.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceBlockingStub.json index f35a058755a..9f037e18413 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceBlockingStub.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceBlockingStub.json @@ -4,7 +4,6 @@ "bindTableToVariable": ":param request: io.deephaven.proto.backplane.script.grpc.BindTableToVariableRequest\n:return: io.deephaven.proto.backplane.script.grpc.BindTableToVariableResponse", "cancelCommand": ":param request: io.deephaven.proto.backplane.script.grpc.CancelCommandRequest\n:return: io.deephaven.proto.backplane.script.grpc.CancelCommandResponse", "executeCommand": ":param request: io.deephaven.proto.backplane.script.grpc.ExecuteCommandRequest\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", - "fetchFigure": ":param request: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", "getConsoleTypes": ":param request: io.deephaven.proto.backplane.script.grpc.GetConsoleTypesRequest\n:return: io.deephaven.proto.backplane.script.grpc.GetConsoleTypesResponse", "nextAutoCompleteStream": "Other half of the browser-based implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:return: io.deephaven.proto.backplane.script.grpc.BrowserNextResponse", "openAutoCompleteStream": "Half of the browser-based (browser's can't do bidirectional streams without websockets)\n implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:return: java.util.Iterator", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceFutureStub.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceFutureStub.json index 6f3ed07ef25..099c3ac6db8 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceFutureStub.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceFutureStub.json @@ -4,7 +4,6 @@ "bindTableToVariable": ":param request: io.deephaven.proto.backplane.script.grpc.BindTableToVariableRequest\n:return: com.google.common.util.concurrent.ListenableFuture", "cancelCommand": ":param request: io.deephaven.proto.backplane.script.grpc.CancelCommandRequest\n:return: com.google.common.util.concurrent.ListenableFuture", "executeCommand": ":param request: io.deephaven.proto.backplane.script.grpc.ExecuteCommandRequest\n:return: com.google.common.util.concurrent.ListenableFuture", - "fetchFigure": ":param request: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n:return: com.google.common.util.concurrent.ListenableFuture", "getConsoleTypes": ":param request: io.deephaven.proto.backplane.script.grpc.GetConsoleTypesRequest\n:return: com.google.common.util.concurrent.ListenableFuture", "nextAutoCompleteStream": "Other half of the browser-based implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:return: com.google.common.util.concurrent.ListenableFuture", "startConsole": ":param request: io.deephaven.proto.backplane.script.grpc.StartConsoleRequest\n:return: com.google.common.util.concurrent.ListenableFuture" diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceImplBase.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceImplBase.json index cd70cca2045..3ae687d6e0b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceImplBase.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceImplBase.json @@ -6,7 +6,6 @@ "bindTableToVariable": ":param request: io.deephaven.proto.backplane.script.grpc.BindTableToVariableRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "cancelCommand": ":param request: io.deephaven.proto.backplane.script.grpc.CancelCommandRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "executeCommand": ":param request: io.deephaven.proto.backplane.script.grpc.ExecuteCommandRequest\n:param responseObserver: io.grpc.stub.StreamObserver", - "fetchFigure": ":param request: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "getConsoleTypes": ":param request: io.deephaven.proto.backplane.script.grpc.GetConsoleTypesRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "nextAutoCompleteStream": "Other half of the browser-based implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "openAutoCompleteStream": "Half of the browser-based (browser's can't do bidirectional streams without websockets)\n implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:param responseObserver: io.grpc.stub.StreamObserver", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceStub.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceStub.json index bdfb0ee560f..9e845060cf7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceStub.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ConsoleServiceGrpc/ConsoleServiceStub.json @@ -5,7 +5,6 @@ "bindTableToVariable": ":param request: io.deephaven.proto.backplane.script.grpc.BindTableToVariableRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "cancelCommand": ":param request: io.deephaven.proto.backplane.script.grpc.CancelCommandRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "executeCommand": ":param request: io.deephaven.proto.backplane.script.grpc.ExecuteCommandRequest\n:param responseObserver: io.grpc.stub.StreamObserver", - "fetchFigure": ":param request: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "getConsoleTypes": ":param request: io.deephaven.proto.backplane.script.grpc.GetConsoleTypesRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "nextAutoCompleteStream": "Other half of the browser-based implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "openAutoCompleteStream": "Half of the browser-based (browser's can't do bidirectional streams without websockets)\n implementation for AutoCompleteStream.\n \n\n:param request: io.deephaven.proto.backplane.script.grpc.AutoCompleteRequest\n:param responseObserver: io.grpc.stub.StreamObserver", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse.json index e08ab08c9bb..6b7da4f2d77 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse.json @@ -2,29 +2,17 @@ "className": "io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", "methods": { "equals": ":param obj: java.lang.Object\n:return: boolean", - "getCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getCreatedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: int", - "getCreatedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", - "getCreatedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getCreatedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", + "getChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldsChangeUpdate) The changes.", + "getChangesOrBuilder": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: io.deephaven.proto.backplane.grpc.FieldsChangeUpdateOrBuilder", "getDefaultInstance": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", "getErrorMessage": "string error_message = 1;\n\n:return: (java.lang.String) The errorMessage.", "getErrorMessageBytes": "string error_message = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for errorMessage.", "getParserForType": ":return: com.google.protobuf.Parser", - "getRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getRemovedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: int", - "getRemovedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", - "getRemovedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getRemovedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", "getSerializedSize": ":return: int", "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "getUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getUpdatedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: int", - "getUpdatedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List", - "getUpdatedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getUpdatedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List", + "hasChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: (boolean) Whether the changes field is set.", "hashCode": ":return: int", "isInitialized": ":return: boolean", "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse/Builder.json index 3b31d28363e..52df706161a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse/Builder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponse/Builder.json @@ -1,66 +1,34 @@ { "className": "io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse$Builder", "methods": { - "addAllCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param values: java.lang.Iterable\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addAllRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param values: java.lang.Iterable\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addAllUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param values: java.lang.Iterable\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 3* \n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 4* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addCreatedBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 2* \n :param index: int\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "addRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 3* \n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 4* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addRemovedBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 2* \n :param index: int\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 3* \n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 4* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "addUpdatedBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 2* \n :param index: int\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", "build": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", "buildPartial": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", "clear": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "clearCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", + "clearChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "clearErrorMessage": "string error_message = 1;\n\n:return: (io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder) This builder for chaining.", "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "clearRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "clearUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "clone": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "getCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getCreatedBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "getCreatedBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", - "getCreatedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: int", - "getCreatedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", - "getCreatedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getCreatedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", + "getChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldsChangeUpdate) The changes.", + "getChangesBuilder": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: io.deephaven.proto.backplane.grpc.FieldsChangeUpdate.Builder", + "getChangesOrBuilder": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: io.deephaven.proto.backplane.grpc.FieldsChangeUpdateOrBuilder", "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", "getErrorMessage": "string error_message = 1;\n\n:return: (java.lang.String) The errorMessage.", "getErrorMessageBytes": "string error_message = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for errorMessage.", - "getRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getRemovedBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "getRemovedBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", - "getRemovedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: int", - "getRemovedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", - "getRemovedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getRemovedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", - "getUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getUpdatedBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "getUpdatedBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List", - "getUpdatedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: int", - "getUpdatedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List", - "getUpdatedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getUpdatedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List", + "hasChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: (boolean) Whether the changes field is set.", "isInitialized": ":return: boolean", + "mergeChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:param value: io.deephaven.proto.backplane.grpc.FieldsChangeUpdate\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "removeCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "removeRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "removeUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "setCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n*Overload 1* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", + "setChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.FieldsChangeUpdate\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.FieldsChangeUpdate.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "setErrorMessage": "string error_message = 1;\n\n:param value: (java.lang.String) - The errorMessage to set.\n:return: (io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder) This builder for chaining.", "setErrorMessageBytes": "string error_message = 1;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for errorMessage to set.\n:return: (io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder) This builder for chaining.", "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "setRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n*Overload 1* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", - "setUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n*Overload 1* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder\n \n*Overload 2* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n :return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder" + "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder" }, "path": "io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse.Builder", "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponse", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponseOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponseOrBuilder.json index e3b0d20b95d..d381fc93168 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponseOrBuilder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/ExecuteCommandResponseOrBuilder.json @@ -1,23 +1,11 @@ { "className": "io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponseOrBuilder", "methods": { - "getCreated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getCreatedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: int", - "getCreatedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", - "getCreatedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getCreatedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition created = 2;\n\n:return: java.util.List", + "getChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: (io.deephaven.proto.backplane.grpc.FieldsChangeUpdate) The changes.", + "getChangesOrBuilder": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: io.deephaven.proto.backplane.grpc.FieldsChangeUpdateOrBuilder", "getErrorMessage": "string error_message = 1;\n\n:return: (java.lang.String) The errorMessage.", "getErrorMessageBytes": "string error_message = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for errorMessage.", - "getRemoved": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getRemovedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: int", - "getRemovedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", - "getRemovedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getRemovedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition removed = 4;\n\n:return: java.util.List", - "getUpdated": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getUpdatedCount": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: int", - "getUpdatedList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List", - "getUpdatedOrBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "getUpdatedOrBuilderList": "repeated .io.deephaven.proto.backplane.script.grpc.VariableDefinition updated = 3;\n\n:return: java.util.List" + "hasChanges": ".io.deephaven.proto.backplane.grpc.FieldsChangeUpdate changes = 2;\n\n:return: (boolean) Whether the changes field is set." }, "path": "io.deephaven.proto.backplane.script.grpc.ExecuteCommandResponseOrBuilder", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest.json deleted file mode 100644 index b8179023f7b..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest.json +++ /dev/null @@ -1,27 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The sourceId.", - "getSourceIdOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hasSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: (boolean) Whether the sourceId field is set.", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest/Builder.json deleted file mode 100644 index 08cde84c861..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequest/Builder.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.FetchFigureRequest$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "build": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "buildPartial": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "clear": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "clearSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "clone": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The sourceId.", - "getSourceIdBuilder": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.Ticket.Builder", - "getSourceIdOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "hasSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: (boolean) Whether the sourceId field is set.", - "isInitialized": ":return: boolean", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "mergeSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:param value: io.deephaven.proto.backplane.grpc.Ticket\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "setSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.Ticket\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.Ticket.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder" - }, - "path": "io.deephaven.proto.backplane.script.grpc.FetchFigureRequest.Builder", - "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.FetchFigureRequest", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequestOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequestOrBuilder.json deleted file mode 100644 index 5d06852de68..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureRequestOrBuilder.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.FetchFigureRequestOrBuilder", - "methods": { - "getSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The sourceId.", - "getSourceIdOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "hasSourceId": ".io.deephaven.proto.backplane.grpc.Ticket source_id = 1;\n\n:return: (boolean) Whether the sourceId field is set." - }, - "path": "io.deephaven.proto.backplane.script.grpc.FetchFigureRequestOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse.json deleted file mode 100644 index c2fc615e0ae..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse.json +++ /dev/null @@ -1,27 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor) The figureDescriptor.", - "getFigureDescriptorOrBuilder": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptorOrBuilder", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hasFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: (boolean) Whether the figureDescriptor field is set.", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse/Builder.json deleted file mode 100644 index 51882e8f934..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponse/Builder.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.FetchFigureResponse$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "build": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "buildPartial": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "clear": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "clearFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "clone": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor) The figureDescriptor.", - "getFigureDescriptorBuilder": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", - "getFigureDescriptorOrBuilder": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptorOrBuilder", - "hasFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: (boolean) Whether the figureDescriptor field is set.", - "isInitialized": ":return: boolean", - "mergeFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:param value: io.deephaven.proto.backplane.script.grpc.FigureDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "setFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.script.grpc.FigureDescriptor\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder" - }, - "path": "io.deephaven.proto.backplane.script.grpc.FetchFigureResponse.Builder", - "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.FetchFigureResponse", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponseOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponseOrBuilder.json deleted file mode 100644 index c0d4fad7a50..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FetchFigureResponseOrBuilder.json +++ /dev/null @@ -1,10 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.FetchFigureResponseOrBuilder", - "methods": { - "getFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor) The figureDescriptor.", - "getFigureDescriptorOrBuilder": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptorOrBuilder", - "hasFigureDescriptor": ".io.deephaven.proto.backplane.script.grpc.FigureDescriptor figure_descriptor = 1;\n\n:return: (boolean) Whether the figureDescriptor field is set." - }, - "path": "io.deephaven.proto.backplane.script.grpc.FetchFigureResponseOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor.json index 74b8e6d522d..d49eaf4e845 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor.json @@ -18,11 +18,6 @@ "getParserForType": ":return: com.google.protobuf.Parser", "getRows": "int32 rows = 9;\n\n:return: (int) The rows.", "getSerializedSize": ":return: int", - "getTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse", - "getTablesCount": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: int", - "getTablesList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", - "getTablesOrBuilder": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponseOrBuilder", - "getTablesOrBuilderList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", "getTitle": "optional string title = 1;\n\n:return: (java.lang.String) The title.", "getTitleBytes": "optional string title = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for title.", "getTitleColor": "string title_color = 3;\n\n:return: (java.lang.String) The titleColor.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor/Builder.json index c511d05c0af..115243aa322 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor/Builder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptor/Builder.json @@ -3,14 +3,11 @@ "methods": { "addAllCharts": "repeated .io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor charts = 10;\n\n:param values: java.lang.Iterable\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "addAllErrors": "repeated string errors = 13;\n\n:param values: (java.lang.Iterable) - The errors to add.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", - "addAllTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param values: java.lang.Iterable\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "addCharts": "repeated .io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor charts = 10;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 2* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 3* \n :param builderForValue: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 4* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "addChartsBuilder": "repeated .io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor charts = 10;\n\n*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor.Builder\n \n*Overload 2* \n :param index: int\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor.Builder", "addErrors": "repeated string errors = 13;\n\n:param value: (java.lang.String) - The errors to add.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "addErrorsBytes": "repeated string errors = 13;\n\n:param value: (com.google.protobuf.ByteString) - The bytes of the errors to add.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", - "addTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 2* \n :param index: int\n :param value: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 3* \n :param builderForValue: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 4* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", - "addTablesBuilder": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n*Overload 1* \n :return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.Builder\n \n*Overload 2* \n :param index: int\n :return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.Builder", "build": ":return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor", "buildPartial": ":return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor", "clear": ":return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", @@ -20,7 +17,6 @@ "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "clearRows": "int32 rows = 9;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", - "clearTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "clearTitle": "optional string title = 1;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "clearTitleColor": "string title_color = 3;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "clearTitleFont": "string title_font = 2;\n\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", @@ -42,13 +38,6 @@ "getErrorsCount": "repeated string errors = 13;\n\n:return: (int) The count of errors.", "getErrorsList": "repeated string errors = 13;\n\n:return: (com.google.protobuf.ProtocolStringList) A list containing the errors.", "getRows": "int32 rows = 9;\n\n:return: (int) The rows.", - "getTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse", - "getTablesBuilder": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.Builder", - "getTablesBuilderList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", - "getTablesCount": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: int", - "getTablesList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", - "getTablesOrBuilder": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponseOrBuilder", - "getTablesOrBuilderList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", "getTitle": "optional string title = 1;\n\n:return: (java.lang.String) The title.", "getTitleBytes": "optional string title = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for title.", "getTitleColor": "string title_color = 3;\n\n:return: (java.lang.String) The titleColor.", @@ -61,14 +50,12 @@ "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.script.grpc.FigureDescriptor\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "removeCharts": "repeated .io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor charts = 10;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", - "removeTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "setCharts": "repeated .io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor charts = 10;\n\n*Overload 1* \n :param index: int\n :param value: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 2* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.ChartDescriptor.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "setCols": "int32 cols = 8;\n\n:param value: (int) - The cols to set.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "setErrors": "repeated string errors = 13;\n\n:param index: (int) - The index to set the value at.\n:param value: (java.lang.String) - The errors to set.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "setRows": "int32 rows = 9;\n\n:param value: (int) - The rows to set.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", - "setTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n*Overload 1* \n :param index: int\n :param value: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder\n \n*Overload 2* \n :param index: int\n :param builderForValue: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.Builder\n :return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder", "setTitle": "optional string title = 1;\n\n:param value: (java.lang.String) - The title to set.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "setTitleBytes": "optional string title = 1;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for title to set.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", "setTitleColor": "string title_color = 3;\n\n:param value: (java.lang.String) - The titleColor to set.\n:return: (io.deephaven.proto.backplane.script.grpc.FigureDescriptor.Builder) This builder for chaining.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptorOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptorOrBuilder.json index c56ed31bf92..f703d01aa85 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptorOrBuilder.json +++ b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/FigureDescriptorOrBuilder.json @@ -12,11 +12,6 @@ "getErrorsCount": "repeated string errors = 13;\n\n:return: (int) The count of errors.", "getErrorsList": "repeated string errors = 13;\n\n:return: (java.util.List) A list containing the errors.", "getRows": "int32 rows = 9;\n\n:return: (int) The rows.", - "getTables": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse", - "getTablesCount": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: int", - "getTablesList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", - "getTablesOrBuilder": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:param index: int\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponseOrBuilder", - "getTablesOrBuilderList": "repeated .io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse tables = 11;\n\n:return: java.util.List", "getTitle": "optional string title = 1;\n\n:return: (java.lang.String) The title.", "getTitleBytes": "optional string title = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for title.", "getTitleColor": "string title_color = 3;\n\n:return: (java.lang.String) The titleColor.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition.json deleted file mode 100644 index 4f70ffe5127..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition.json +++ /dev/null @@ -1,31 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "methods": { - "equals": ":param obj: java.lang.Object\n:return: boolean", - "getDefaultInstance": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The id.", - "getIdOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "getParserForType": ":return: com.google.protobuf.Parser", - "getSerializedSize": ":return: int", - "getTitle": "string title = 2;\n\n:return: (java.lang.String) The title.", - "getTitleBytes": "string title = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for title.", - "getType": "string type = 1;\n\n:return: (java.lang.String) The type.", - "getTypeBytes": "string type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", - "getUnknownFields": ":return: com.google.protobuf.UnknownFieldSet", - "hasId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: (boolean) Whether the id field is set.", - "hashCode": ":return: int", - "isInitialized": ":return: boolean", - "newBuilder": "*Overload 1* \n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 2* \n :param prototype: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "newBuilderForType": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "parseDelimitedFrom": "*Overload 1* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 2* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "parseFrom": "*Overload 1* \n :param data: java.nio.ByteBuffer\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 2* \n :param data: java.nio.ByteBuffer\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 3* \n :param data: com.google.protobuf.ByteString\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 4* \n :param data: com.google.protobuf.ByteString\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 5* \n :param data: byte[]\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 6* \n :param data: byte[]\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 7* \n :param input: java.io.InputStream\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 8* \n :param input: java.io.InputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 9* \n :param input: com.google.protobuf.CodedInputStream\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n \n*Overload 10* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "parser": ":return: com.google.protobuf.Parser", - "toBuilder": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "writeTo": ":param output: com.google.protobuf.CodedOutputStream" - }, - "path": "io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition/Builder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition/Builder.json deleted file mode 100644 index bd4ca3b42c5..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinition/Builder.json +++ /dev/null @@ -1,41 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.VariableDefinition$Builder", - "methods": { - "addRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "build": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "buildPartial": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "clear": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "clearField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "clearId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "clearOneof": ":param oneof: com.google.protobuf.Descriptors.OneofDescriptor\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "clearTitle": "string title = 2;\n\n:return: (io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder) This builder for chaining.", - "clearType": "string type = 1;\n\n:return: (io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder) This builder for chaining.", - "clone": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "getDefaultInstanceForType": ":return: io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "getDescriptor": ":return: com.google.protobuf.Descriptors.Descriptor", - "getDescriptorForType": ":return: com.google.protobuf.Descriptors.Descriptor", - "getId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The id.", - "getIdBuilder": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: io.deephaven.proto.backplane.grpc.Ticket.Builder", - "getIdOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "getTitle": "string title = 2;\n\n:return: (java.lang.String) The title.", - "getTitleBytes": "string title = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for title.", - "getType": "string type = 1;\n\n:return: (java.lang.String) The type.", - "getTypeBytes": "string type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", - "hasId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: (boolean) Whether the id field is set.", - "isInitialized": ":return: boolean", - "mergeFrom": "*Overload 1* \n :param other: com.google.protobuf.Message\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 2* \n :param other: io.deephaven.proto.backplane.script.grpc.VariableDefinition\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 3* \n :param input: com.google.protobuf.CodedInputStream\n :param extensionRegistry: com.google.protobuf.ExtensionRegistryLite\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "mergeId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:param value: io.deephaven.proto.backplane.grpc.Ticket\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "mergeUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "setField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "setId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n*Overload 1* \n :param value: io.deephaven.proto.backplane.grpc.Ticket\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder\n \n*Overload 2* \n :param builderForValue: io.deephaven.proto.backplane.grpc.Ticket.Builder\n :return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "setRepeatedField": ":param field: com.google.protobuf.Descriptors.FieldDescriptor\n:param index: int\n:param value: java.lang.Object\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "setTitle": "string title = 2;\n\n:param value: (java.lang.String) - The title to set.\n:return: (io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder) This builder for chaining.", - "setTitleBytes": "string title = 2;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for title to set.\n:return: (io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder) This builder for chaining.", - "setType": "string type = 1;\n\n:param value: (java.lang.String) - The type to set.\n:return: (io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder) This builder for chaining.", - "setTypeBytes": "string type = 1;\n\n:param value: (com.google.protobuf.ByteString) - The bytes for type to set.\n:return: (io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder) This builder for chaining.", - "setUnknownFields": ":param unknownFields: com.google.protobuf.UnknownFieldSet\n:return: io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder" - }, - "path": "io.deephaven.proto.backplane.script.grpc.VariableDefinition.Builder", - "text": "Protobuf type io.deephaven.proto.backplane.script.grpc.VariableDefinition", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinitionOrBuilder.json b/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinitionOrBuilder.json deleted file mode 100644 index 15ab2b20731..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/proto/backplane/script/grpc/VariableDefinitionOrBuilder.json +++ /dev/null @@ -1,14 +0,0 @@ -{ - "className": "io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "methods": { - "getId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: (io.deephaven.proto.backplane.grpc.Ticket) The id.", - "getIdOrBuilder": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: io.deephaven.proto.backplane.grpc.TicketOrBuilder", - "getTitle": "string title = 2;\n\n:return: (java.lang.String) The title.", - "getTitleBytes": "string title = 2;\n\n:return: (com.google.protobuf.ByteString) The bytes for title.", - "getType": "string type = 1;\n\n:return: (java.lang.String) The type.", - "getTypeBytes": "string type = 1;\n\n:return: (com.google.protobuf.ByteString) The bytes for type.", - "hasId": ".io.deephaven.proto.backplane.grpc.Ticket id = 3;\n\n:return: (boolean) Whether the id field is set." - }, - "path": "io.deephaven.proto.backplane.script.grpc.VariableDefinitionOrBuilder", - "typeName": "interface" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/appmode/ScriptApplicationState.json b/Integrations/python/deephaven/doc/io/deephaven/server/appmode/ScriptApplicationState.json index fd598c438d4..5d54ec02b66 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/server/appmode/ScriptApplicationState.json +++ b/Integrations/python/deephaven/doc/io/deephaven/server/appmode/ScriptApplicationState.json @@ -1,7 +1,6 @@ { "className": "io.deephaven.server.appmode.ScriptApplicationState", "methods": { - "setCustomField": "*Overload 1* \n Note: Java generics information - \n \n :param type: java.lang.String\n :param name: java.lang.String\n :param value: T\n \n*Overload 2* \n Note: Java generics information - \n \n :param type: java.lang.String\n :param name: java.lang.String\n :param value: T\n :param description: java.lang.String", "setField": "*Overload 1* \n Note: Java generics information - \n \n :param name: java.lang.String\n :param value: T\n :param description: java.lang.String\n \n*Overload 2* \n Note: Java generics information - \n \n :param name: java.lang.String\n :param value: T" }, "path": "io.deephaven.server.appmode.ScriptApplicationState", diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/IndexGenerator.json b/Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/RowSetGenerator.json similarity index 68% rename from Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/IndexGenerator.json rename to Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/RowSetGenerator.json index 437a280b65f..e467fba32a5 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/IndexGenerator.json +++ b/Integrations/python/deephaven/doc/io/deephaven/server/barrage/BarrageStreamGenerator/RowSetGenerator.json @@ -1,8 +1,8 @@ { - "className": "io.deephaven.server.barrage.BarrageStreamGenerator$IndexGenerator", + "className": "io.deephaven.server.barrage.BarrageStreamGenerator$RowSetGenerator", "methods": { "getInputStream": ":return: io.deephaven.server.barrage.BarrageStreamGenerator.DrainableByteArrayInputStream" }, - "path": "io.deephaven.server.barrage.BarrageStreamGenerator.IndexGenerator", + "path": "io.deephaven.server.barrage.BarrageStreamGenerator.RowSetGenerator", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/console/ConsoleServiceGrpcImpl.json b/Integrations/python/deephaven/doc/io/deephaven/server/console/ConsoleServiceGrpcImpl.json index 54ad277657f..8e81c78eec4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/server/console/ConsoleServiceGrpcImpl.json +++ b/Integrations/python/deephaven/doc/io/deephaven/server/console/ConsoleServiceGrpcImpl.json @@ -5,8 +5,8 @@ "bindTableToVariable": ":param request: io.deephaven.proto.backplane.script.grpc.BindTableToVariableRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "cancelCommand": ":param request: io.deephaven.proto.backplane.script.grpc.CancelCommandRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "executeCommand": ":param request: io.deephaven.proto.backplane.script.grpc.ExecuteCommandRequest\n:param responseObserver: io.grpc.stub.StreamObserver", - "fetchFigure": ":param request: io.deephaven.proto.backplane.script.grpc.FetchFigureRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "getConsoleTypes": ":param request: io.deephaven.proto.backplane.script.grpc.GetConsoleTypesRequest\n:param responseObserver: io.grpc.stub.StreamObserver", + "isPythonSession": ":return: boolean", "startConsole": ":param request: io.deephaven.proto.backplane.script.grpc.StartConsoleRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "subscribeToLogs": ":param request: io.deephaven.proto.backplane.script.grpc.LogSubscriptionRequest\n:param responseObserver: io.grpc.stub.StreamObserver" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/console/figure/FigureWidgetTranslator.json b/Integrations/python/deephaven/doc/io/deephaven/server/console/figure/FigureWidgetTranslator.json deleted file mode 100644 index 2075dd475ba..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/server/console/figure/FigureWidgetTranslator.json +++ /dev/null @@ -1,8 +0,0 @@ -{ - "className": "io.deephaven.server.console.figure.FigureWidgetTranslator", - "methods": { - "translate": ":param figure: io.deephaven.plot.FigureWidget\n:param sessionState: io.deephaven.server.session.SessionState\n:return: io.deephaven.proto.backplane.script.grpc.FigureDescriptor" - }, - "path": "io.deephaven.server.console.figure.FigureWidgetTranslator", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceGrpcImpl.json b/Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceGrpcImpl.json new file mode 100644 index 00000000000..df8e4db129d --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceGrpcImpl.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.server.object.ObjectServiceGrpcImpl", + "methods": { + "fetchObject": ":param request: io.deephaven.proto.backplane.grpc.FetchObjectRequest\n:param responseObserver: io.grpc.stub.StreamObserver" + }, + "path": "io.deephaven.server.object.ObjectServiceGrpcImpl", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceModule.json b/Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceModule.json new file mode 100644 index 00000000000..b390c0dcf99 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/object/ObjectServiceModule.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.server.object.ObjectServiceModule", + "methods": { + "bindObjectServiceGrpcImpl": ":param objectService: io.deephaven.server.object.ObjectServiceGrpcImpl\n:return: io.grpc.BindableService" + }, + "path": "io.deephaven.server.object.ObjectServiceModule", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/object/TypeLookup.json b/Integrations/python/deephaven/doc/io/deephaven/server/object/TypeLookup.json new file mode 100644 index 00000000000..12141a80f34 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/object/TypeLookup.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.server.object.TypeLookup", + "methods": { + "type": ":param object: java.lang.Object\n:return: java.util.Optional" + }, + "path": "io.deephaven.server.object.TypeLookup", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsAutoDiscovery.json b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsAutoDiscovery.json new file mode 100644 index 00000000000..ef576b85eac --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsAutoDiscovery.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.server.plugin.PluginsAutoDiscovery", + "methods": { + "registerAll": "Registers plugins via JavaServiceLoader.loadAllAndRegisterInto(Callback) and\n PythonModuleLoader.allRegisterInto(Callback) (if python is enabled).\n\n*Overload 1* \n \n \n*Overload 2* \n :param includePython: boolean" + }, + "path": "io.deephaven.server.plugin.PluginsAutoDiscovery", + "text": "Provides a registerAll() entrypoint for Registration auto-discovery. Logs auto-discovered details.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsModule.json b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsModule.json new file mode 100644 index 00000000000..dfa8b7bf97f --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/PluginsModule.json @@ -0,0 +1,9 @@ +{ + "className": "io.deephaven.server.plugin.PluginsModule", + "methods": { + "bindPluginRegistrationCallback": ":param visitor: io.deephaven.server.plugin.PluginRegistrationVisitor\n:return: io.deephaven.plugin.Registration.Callback" + }, + "path": "io.deephaven.server.plugin.PluginsModule", + "text": "Includes the modules necessary to provide PluginsAutoDiscovery.\n\n \n Note: runtime plugin registration is not currently supported - ie, no Registration.Callback is provided. See\n deephaven-core#1809 for the feature request.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/plugin/java/JavaServiceLoader.json b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/java/JavaServiceLoader.json new file mode 100644 index 00000000000..8b9e1ff4b84 --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/java/JavaServiceLoader.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.server.plugin.java.JavaServiceLoader", + "methods": { + "loadAllAndRegisterInto": "Load all registrations found via ServiceLoader.load(Class), and register them into\n callback.\n\n:param callback: (io.deephaven.plugin.Registration.Callback) - the plugin callback" + }, + "path": "io.deephaven.server.plugin.java.JavaServiceLoader", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/plugin/python/PythonModuleLoader.json b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/python/PythonModuleLoader.json new file mode 100644 index 00000000000..02ff57fe17b --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/python/PythonModuleLoader.json @@ -0,0 +1,8 @@ +{ + "className": "io.deephaven.server.plugin.python.PythonModuleLoader", + "methods": { + "allRegisterInto": "Registers all plugins found via python method \"deephaven.plugin:register_all_into\". See the\n deephaven-plugin python package for more information.\n\n:param callback: (io.deephaven.plugin.Registration.Callback) - the plugin callback" + }, + "path": "io.deephaven.server.plugin.python.PythonModuleLoader", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypes.json b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypes.json new file mode 100644 index 00000000000..378d464eeaf --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypes.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.server.plugin.type.ObjectTypes", + "methods": { + "findObjectType": "Find the ObjectType compatible with object. That is, ObjectType.isType(Object) will be\n true for object.\n\n:param object: (java.lang.Object) - the object\n:return: (java.util.Optional) the object type, if found", + "register": "Register objectType.\n\n:param objectType: (io.deephaven.plugin.type.ObjectType) - the object type" + }, + "path": "io.deephaven.server.plugin.type.ObjectTypes", + "text": "Provides synchronized object type registration and lookup.\n\n \n Object type registration that is an instances of ObjectTypeClassBase receives special consideration, and\n these objects have more efficient lookups.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypesModule.json b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypesModule.json new file mode 100644 index 00000000000..4f933ad390d --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/server/plugin/type/ObjectTypesModule.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.server.plugin.type.ObjectTypesModule", + "methods": { + "bindsCallback": ":param types: io.deephaven.server.plugin.type.ObjectTypes\n:return: io.deephaven.plugin.type.ObjectTypeRegistration", + "bindsLookup": ":param types: io.deephaven.server.plugin.type.ObjectTypes\n:return: io.deephaven.plugin.type.ObjectTypeLookup" + }, + "path": "io.deephaven.server.plugin.type.ObjectTypesModule", + "text": "Binds ObjectTypes as ObjectTypeLookup and ObjectTypeRegistration.", + "typeName": "interface" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/server/table/TableServiceGrpcImpl.json b/Integrations/python/deephaven/doc/io/deephaven/server/table/TableServiceGrpcImpl.json index 4259d40c383..8044a182c4b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/server/table/TableServiceGrpcImpl.json +++ b/Integrations/python/deephaven/doc/io/deephaven/server/table/TableServiceGrpcImpl.json @@ -4,7 +4,6 @@ "applyPreviewColumns": "Create a table that has preview columns applied to an existing source table.\n \n\n:param request: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "asOfJoinTables": "Returns the result of an as of join operation.\n \n\n:param request: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "batch": "Batch a series of requests and send them all at once. This enables the user to create intermediate tables without\n requiring them to be exported and managed by the client. The server will automatically release any tables when they\n are no longer depended upon.\n \n\n:param request: io.deephaven.proto.backplane.grpc.BatchTableRequest\n:param responseObserver: io.grpc.stub.StreamObserver", - "buildTableCreationResponse": ":param tableRef: io.deephaven.proto.backplane.grpc.TableReference\n:param table: io.deephaven.engine.table.Table\n:return: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse", "comboAggregate": "Returns the result of an aggregate table operation.\n \n\n:param request: io.deephaven.proto.backplane.grpc.ComboAggregateRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "createInputTable": "Creates a new Table based on the provided configuration. This can be used as a regular table from the other methods\n in this interface, or can be interacted with via the InputTableService to modify its contents.\n \n\n:param request: io.deephaven.proto.backplane.grpc.CreateInputTableRequest\n:param responseObserver: io.grpc.stub.StreamObserver", "crossJoinTables": "Returns the result of a cross join operation. Also known as the cartesian product.\n \n\n:param request: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest\n:param responseObserver: io.grpc.stub.StreamObserver", diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index f509948dd21..27384b1180d 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -82,7 +82,8 @@ private static AggregationContextFactory makeGroupByACF( return AggregationProcessor.forAggregation(List.of( AggregateAllByTable.singleAggregation(AggSpec.group(), Selectable.from(groupByColumns), table.getDefinition().getColumnStream().map(ColumnDefinition::getName) - .map(ColumnName::of).collect(Collectors.toList())).orElseThrow())); + .map(ColumnName::of).collect(Collectors.toList())) + .orElseThrow())); } private static Table individualStaticByTest(@NotNull final Table input, From 59e08de55afe5581b3c555576af8f65f0c9d7a5e Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 31 Jan 2022 00:42:48 -0500 Subject: [PATCH 32/44] Fix unit tests --- .../table/impl/by/AggregationProcessor.java | 72 +++++++++++++------ .../by/rollup/RollupAggregationPairs.java | 42 +++++++++++ .../engine/table/impl/QueryFactory.java | 4 +- .../table/impl/QueryTableAggregationTest.java | 4 +- .../engine/table/impl/QueryTableTreeTest.java | 9 +-- .../engine/table/impl/TestAggBy.java | 21 +++--- .../client/impl/BatchTableRequestBuilder.java | 22 +++--- .../io/deephaven/api/agg/Aggregation.java | 21 ++++-- .../api/agg/AggregationDescriptions.java | 5 ++ .../api/agg/AggregationOptimizer.java | 5 ++ .../deephaven/api/agg/AggregationPairs.java | 9 ++- .../io/deephaven/api/agg/util/Sentinel.java | 32 +++++++++ 12 files changed, 192 insertions(+), 54 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/util/Sentinel.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index b42d28659e4..25376695e23 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -3,6 +3,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.SortColumn; import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.Aggregations; import io.deephaven.api.agg.AggregationPairs; import io.deephaven.api.agg.ColumnAggregation; import io.deephaven.api.agg.ColumnAggregations; @@ -50,6 +51,7 @@ import io.deephaven.engine.table.impl.by.rollup.NullColumns; import io.deephaven.engine.table.impl.by.rollup.Partition; import io.deephaven.engine.table.impl.by.rollup.RollupAggregation; +import io.deephaven.engine.table.impl.by.rollup.RollupAggregationPairs; import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ByteChunkedCountDistinctOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.ByteRollupCountDistinctOperator; import io.deephaven.engine.table.impl.by.ssmcountdistinct.count.CharChunkedCountDistinctOperator; @@ -154,7 +156,16 @@ public class AggregationProcessor implements AggregationContextFactory { private enum Type { - NORMAL, ROLLUP_BASE, ROLLUP_REAGGREGATED, SELECT_DISTINCT + NORMAL(false), + ROLLUP_BASE(true), + ROLLUP_REAGGREGATED(true), + SELECT_DISTINCT(false); + + private final boolean isRollup; + + Type(boolean isRollup) { + this.isRollup = isRollup; + } } private final Collection aggregations; @@ -227,10 +238,13 @@ private AggregationProcessor( @NotNull final Type type) { this.aggregations = aggregations; this.type = type; - final String duplicationErrorMessage = AggregationPairs.outputsOf(aggregations) - .collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream() - .filter(kv -> kv.getValue() > 1).map(kv -> kv.getKey() + " used " + kv.getValue() + " times") - .collect(Collectors.joining(", ")); + final String duplicationErrorMessage = + (type.isRollup + ? RollupAggregationPairs.outputsOf(aggregations) + : AggregationPairs.outputsOf(aggregations)) + .collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream() + .filter(kv -> kv.getValue() > 1).map(kv -> kv.getKey() + " used " + kv.getValue() + " times") + .collect(Collectors.joining(", ")); if (!duplicationErrorMessage.isBlank()) { throw new IllegalArgumentException("Duplicate output columns found: " + duplicationErrorMessage); } @@ -295,9 +309,18 @@ private Converter(@NotNull final Table table, @NotNull final String... groupByCo } AggregationContext build() { + walkAllAggregations(); + return makeAggregationContext(); + } + + final void walkAllAggregations() { for (final Aggregation aggregation : aggregations) { aggregation.walk(this); } + } + + @NotNull + final AggregationContext makeAggregationContext() { // noinspection unchecked return new AggregationContext( operators.toArray(IterativeChunkedAggregationOperator[]::new), @@ -306,7 +329,7 @@ AggregationContext build() { transformers.toArray(AggregationContextTransformer[]::new)); } - void streamUnsupported(@NotNull final String operationName) { + final void streamUnsupported(@NotNull final String operationName) { if (!isStream) { return; } @@ -319,6 +342,11 @@ void streamUnsupported(@NotNull final String operationName) { // Partial Aggregation.Visitor (for cases common to all types) // ------------------------------------------------------------------------------------------------------------- + @Override + public final void visit(@NotNull final Aggregations aggregations) { + aggregations.aggregations().forEach(a -> a.walk(this)); + } + @Override public final void visit(@NotNull final ColumnAggregation columnAgg) { resultPairs = List.of(columnAgg.pair()); @@ -840,10 +868,11 @@ private RollupBaseConverter(@NotNull final Table table, @NotNull final String... @Override AggregationContext build() { + walkAllAggregations(); if (!partitionFound) { transformers.add(new NoKeyLeafRollupAttributeSetter()); } - return super.build(); + return makeAggregationContext(); } // ------------------------------------------------------------------------------------------------------------- @@ -983,7 +1012,7 @@ private RollupReaggregatedConverter(@NotNull final Table table, @NotNull final S @Override public void visit(@NotNull final Count count) { - addNoInputOperator(new LongChunkedSumOperator(false, count.column().name())); + addBasicOperators((t, n) -> new LongChunkedSumOperator(false, n)); } @Override @@ -1021,14 +1050,14 @@ public void visit(@NotNull final AggSpecAbsSum absSum) { @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { - reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> makeCountDistinctOperator(ssmType, n, - countDistinct.countNulls(), true, true)); + reaggregateSsmBackedOperator((ssmSrc, priorResultSrc, n) -> makeCountDistinctOperator( + ssmSrc.getComponentType(), n, countDistinct.countNulls(), true, true)); } @Override public void visit(@NotNull final AggSpecDistinct distinct) { - reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> makeDistinctOperator(priorResultType, n, - distinct.includeNulls(), true, true)); + reaggregateSsmBackedOperator((ssmSrc, priorResultSrc, n) -> makeDistinctOperator( + priorResultSrc.getComponentType(), n, distinct.includeNulls(), true, true)); } @Override @@ -1078,8 +1107,8 @@ public void visit(@NotNull final AggSpecSum sum) { @Override public void visit(@NotNull final AggSpecUnique unique) { - reaggregateSsmBackedOperator((ssmType, priorResultType, n) -> makeUniqueOperator( - priorResultType, n, unique.includeNulls(), null, unique.nonUniqueSentinel(), true, true)); + reaggregateSsmBackedOperator((ssmSrc, priorResultSrc, n) -> makeUniqueOperator( + priorResultSrc.getType(), n, unique.includeNulls(), null, unique.nonUniqueSentinel(), true, true)); } @Override @@ -1102,14 +1131,15 @@ private void reaggregateAsSum() { } private void reaggregateSsmBackedOperator( - TriFunction, Class, String, IterativeChunkedAggregationOperator> operatorFactory) { + TriFunction>, ColumnSource, String, + IterativeChunkedAggregationOperator> operatorFactory) { for (final Pair pair : resultPairs) { final String resultName = pair.output().name(); final String ssmName = resultName + ROLLUP_DISTINCT_SSM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; final ColumnSource> ssmSource = table.getColumnSource(ssmName); final ColumnSource priorResultSource = table.getColumnSource(resultName); final IterativeChunkedAggregationOperator operator = operatorFactory.apply( - ssmSource.getComponentType(), priorResultSource.getComponentType(), resultName); + ssmSource, priorResultSource, resultName); addOperator(operator, ssmSource, ssmName); } @@ -1147,10 +1177,11 @@ private void reaggregateAvgOperator() { final String nonNullCountName = resultName + ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; final LongChunkedSumOperator nonNullCountOp = addAndGetLongSumOperator(nonNullCountName); - if (runningSumType == double.class) { + final String nanCountName = resultName + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + + if (table.hasColumns(nanCountName)) { final DoubleChunkedSumOperator runningSumOp = addAndGetDoubleSumOperator(runningSumName); - final String nanCountName = resultName + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; final LongChunkedSumOperator nanCountOp = addAndGetLongSumOperator(nanCountName); final String piCountName = resultName + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; @@ -1197,11 +1228,12 @@ private void reaggregateStdOrVarOperators(final boolean isStd) { final String nonNullCountName = resultName + ROLLUP_NONNULL_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; final LongChunkedSumOperator nonNullCountOp = addAndGetLongSumOperator(nonNullCountName); - if (runningSumType == double.class) { + final String nanCountName = resultName + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; + + if (table.hasColumns(nanCountName)) { final DoubleChunkedSumOperator runningSumOp = addAndGetDoubleSumOperator(runningSumName); final DoubleChunkedSumOperator runningSum2Op = addAndGetDoubleSumOperator(runningSum2Name); - final String nanCountName = resultName + ROLLUP_NAN_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; final LongChunkedSumOperator nanCountOp = addAndGetLongSumOperator(nanCountName); final String piCountName = resultName + ROLLUP_PI_COUNT_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.java new file mode 100644 index 00000000000..7c34be9b657 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.java @@ -0,0 +1,42 @@ +package io.deephaven.engine.table.impl.by.rollup; + +import io.deephaven.api.ColumnName; +import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.AggregationPairs; +import io.deephaven.api.agg.Pair; + +import java.util.Collection; +import java.util.stream.Stream; + +/** + * A visitor to get the ordered input/output {@link Pair column name pairs} for {@link Aggregation aggregations}, + * including {@link RollupAggregation rollup aggregations}. + */ +public class RollupAggregationPairs extends AggregationPairs implements RollupAggregation.Visitor { + + public static Stream of(Aggregation aggregation) { + return aggregation.walk(new RollupAggregationPairs()).getOut(); + } + + public static Stream of(Collection aggregations) { + return aggregations.stream().flatMap(RollupAggregationPairs::of); + } + + public static Stream outputsOf(Aggregation aggregation) { + return of(aggregation).map(Pair::output); + } + + public static Stream outputsOf(Collection aggregations) { + return of(aggregations).map(Pair::output); + } + + @Override + public void visit(NullColumns nullColumns) { + out = nullColumns.resultColumns().keySet().stream().map(ColumnName::of); + } + + @Override + public void visit(Partition partition) { + out = Stream.empty(); + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java index 7ff2389e09b..7fc47671b45 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryFactory.java @@ -808,9 +808,7 @@ private void addNormalOperation(int opNum, StringBuilder opChain, Random random, * @return A string that contains all common things that are needed to use generateQuery() */ public String getTablePreamble(Long tableSeed) { - return "\n\nimport io.deephaven.engine.table.impl.by.SortedFirstBy;\n" + - "import io.deephaven.engine.table.impl.by.SortedLastBy;\n\n\n" + - "import io.deephaven.engine.table.impl.by.PercentileBySpecImpl;\n" + + return "\n\nimport static io.deephaven.api.agg.Aggregation.*;\n" + "tableSeed = " + tableSeed + " as long;\n" + "size = 100 as int;\n" + "scale = 1000 as int;\n" + diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java index 27384b1180d..900bbf92364 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableAggregationTest.java @@ -2537,7 +2537,7 @@ public void testTDigestAccumulation() { TableTools.showWithRowSet(aggregatedBySym); final Table accumulated = aggregatedBySym.dropColumns("Sym").groupBy() - .update("Digest=io.deephaven.engine.table.impl.by.AggSpecTDigest.accumulateDigests(Digest)") + .update("Digest=io.deephaven.engine.table.impl.by.ApproximatePercentile.accumulateDigests(Digest)") .update("P95=Digest.quantile(0.95)"); TableTools.show(accumulated); @@ -2825,7 +2825,7 @@ public void testCountBy() { newTable().countBy("x = 1"); TestCase.fail("should throw an exception"); } catch (RuntimeException e) { - TestCase.assertEquals("x = 1 is not a valid column name", e.getMessage()); + TestCase.assertTrue(e.getMessage().contains("x = 1")); } try { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java index e870492cdec..7bbde09028a 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableTreeTest.java @@ -1296,7 +1296,8 @@ public void testRollupDistinct() { public void testRollupUnique() { testSimpleRollup(List.of(AggUnique("IntCol", "DoubleCol", "FloatNullCol", "StringCol", "BoolCol"))); - testSimpleRollup(List.of(AggUnique(true, null, "IntCol", "DoubleCol", "FloatNullCol", "StringCol", "BoolCol"))); + testSimpleRollup(List.of(AggUnique(true, Sentinel(), "IntCol", "DoubleCol", "FloatNullCol", "StringCol", + "BoolCol"))); } private void testSimpleRollup(Collection comboAgg) { @@ -1548,7 +1549,7 @@ public void testRollupDistinctIncremental() { public void testRollupUniqueIncremental() { testIncrementalSimple(AggUnique("IntCol")); - testIncrementalSimple(AggUnique(true, null, "IntCol")); + testIncrementalSimple(AggUnique(true, Sentinel(), "IntCol")); // TODO (https://github.com/deephaven/deephaven-core/issues/991): Re-enable these sub-tests // testIncrementalSimple(AggUnique(false, -1, -2, "IntCol")); // testIncrementalSimple(AggUnique(true, -1, -2, "IntCol")); @@ -1616,7 +1617,7 @@ public void testDuplicateAgg() { simpleTable.rollup(List.of(AggCount("MyString"), AggMin("MyString")), "MyDouble"); TestCase.fail("No exception generated with duplicate output column names."); } catch (IllegalArgumentException iae) { - assertEquals("Duplicate output columns: MyString used 2 times", iae.getMessage()); + assertEquals("Duplicate output columns found: MyString used 2 times", iae.getMessage()); } } @@ -1679,7 +1680,7 @@ private void testRollupIncremental(int seed) { AggDistinct(true, "SDistinctN=StringNulls", "DistinctBoolColN=BoolCol", "DNIntCol=IntSet", "DNLongCol=LongSet", "DNDoubleCol=DoubleSet", "DNFloatCol=FloatSet", "DNCharCol=CharSet", "DNShortCol=ShortSet", "DNByteCol=ByteSet"), - AggUnique(true, null, "SUniqueN=StringNulls", "UniqueBoolColN=BoolCol", + AggUnique(true, Sentinel(), "SUniqueN=StringNulls", "UniqueBoolColN=BoolCol", "UNIntCol=IntSet", "UNLongCol=LongSet", "UNDoubleCol=DoubleSet", "UNFloatCol=FloatSet", "UNCharCol=CharSet", "UNShortCol=ShortSet", "UNByteCol=ByteSet")); final EvalNuggetInterface[] en = new EvalNuggetInterface[] { diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java index 9c73e57c96f..b638cb05e31 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java @@ -538,12 +538,12 @@ public void testComboByAggUnique() { c("Whee", dt1, dt1, dt1, /**/ dt1, dt2, /**/ dt2, dt2, dt2, dt2, /**/ null)); Table result = dataTable.aggBy(List.of( - AggUnique(false, -1, "Account", "Qty"), - AggUnique(false, dtDefault, "Whee")), "USym").sort("USym"); + AggUnique(false, Sentinel(-1), "Account", "Qty"), + AggUnique(false, Sentinel(dtDefault), "Whee")), "USym").sort("USym"); Table countNulls = dataTable.aggBy(List.of( - AggUnique(true, -1, "Account", "Qty"), - AggUnique(true, dtDefault, "Whee")), "USym").sort("USym"); + AggUnique(true, Sentinel(-1), "Account", "Qty"), + AggUnique(true, Sentinel(dtDefault), "Whee")), "USym").sort("USym"); assertEquals(4, result.size()); assertArrayEquals(new Object[] {"AAPL", -1L, 100, dt1}, result.getRecord(0)); @@ -632,10 +632,11 @@ public void testAggUniqueDefaultValues() { // First try mixing column types and values expectException(IllegalArgumentException.class, "Attempted to use no key/non unique values of incorrect types for aggregated columns!", - () -> dataTable.aggBy(AggUnique(false, 2, "StringCol", "BoolCol", "DatTime", "CharCol", + () -> dataTable.aggBy(AggUnique(false, Sentinel(2), "StringCol", "BoolCol", "DatTime", "CharCol", "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol"), "USym").sort("USym")); - dataTable.aggBy(AggUnique(false, -2, "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol"), + dataTable.aggBy(AggUnique(false, Sentinel(-2), "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", + "DoubleCol"), "USym").sort("USym"); // Byte out of range @@ -657,14 +658,14 @@ private void testUniqueOutOfRangeParams(Class type, Table dataTable, Number i // Byte out of range expectException(IllegalArgumentException.class, "Attempted to non unique values too small for " + type.getName() + "!", - () -> dataTable.aggBy(AggUnique(false, invalidLow, aggCols), "USym").sort("USym")); + () -> dataTable.aggBy(AggUnique(false, Sentinel(invalidLow), aggCols), "USym").sort("USym")); expectException(IllegalArgumentException.class, "Attempted to use non unique values too large for " + type.getName() + "!", - () -> dataTable.aggBy(AggUnique(false, invalidHigh, aggCols), "USym").sort("USym")); + () -> dataTable.aggBy(AggUnique(false, Sentinel(invalidHigh), aggCols), "USym").sort("USym")); - dataTable.aggBy(AggUnique(false, validLow, aggCols), "USym").sort("USym"); - dataTable.aggBy(AggUnique(false, validHigh, aggCols), "USym").sort("USym"); + dataTable.aggBy(AggUnique(false, Sentinel(validLow), aggCols), "USym").sort("USym"); + dataTable.aggBy(AggUnique(false, Sentinel(validHigh), aggCols), "USym").sort("USym"); } private static void expectException(@SuppressWarnings("SameParameterValue") Class excType, diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index 2c3ba896fdd..6287c4b31fd 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -108,6 +108,7 @@ import java.util.OptionalInt; import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; +import java.util.stream.Stream; class BatchTableRequestBuilder { @@ -461,7 +462,7 @@ private ComboAggregateRequest aggAllBy(AggregateAllByTable agg) { private ComboAggregateRequest aggBy(AggregationTable agg) { ComboAggregateRequest.Builder builder = groupByColumns(agg); for (Aggregation aggregation : agg.aggregations()) { - builder.addAggregates(AggregationAdapter.of(aggregation)); + AggregationAdapter.of(aggregation).forEach(builder::addAggregates); } return builder.build(); } @@ -497,19 +498,24 @@ private ComboAggregateRequest countBy(CountByTable countByTable) { private static class AggregationAdapter implements Aggregation.Visitor { - public static Aggregate of(Aggregation aggregation) { + public static Stream of(Aggregation aggregation) { return aggregation.walk(new AggregationAdapter()).out(); } - private Aggregate out; + private Stream out; - public Aggregate out() { + public Stream out() { return Objects.requireNonNull(out); } + @Override + public void visit(Aggregations aggregations) { + out = aggregations.aggregations().stream().flatMap(AggregationAdapter::of); + } + @Override public void visit(Count count) { - out = Aggregate.newBuilder().setType(AggType.COUNT).setColumnName(count.column().name()).build(); + out = Stream.of(Aggregate.newBuilder().setType(AggType.COUNT).setColumnName(count.column().name()).build()); } @Override @@ -526,13 +532,13 @@ public void visit(LastRowKey lastRowKey) { @Override public void visit(ColumnAggregation columnAgg) { - out = columnAgg.spec() - .walk(new AggregateAdapter(Collections.singletonList(columnAgg.pair()))).out(); + out = Stream.of(columnAgg.spec() + .walk(new AggregateAdapter(Collections.singletonList(columnAgg.pair()))).out()); } @Override public void visit(ColumnAggregations columnAggs) { - out = columnAggs.spec().walk(new AggregateAdapter(columnAggs.pairs())).out(); + out = Stream.of(columnAggs.spec().walk(new AggregateAdapter(columnAggs.pairs())).out()); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index cce6b1f5e0f..d579a9c6e0f 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -3,6 +3,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.util.PercentileOutput; +import io.deephaven.api.agg.util.Sentinel; import java.io.Serializable; import java.util.Collection; @@ -203,8 +204,12 @@ static Aggregation AggUnique(String... pairs) { return of(AggSpec.unique(), pairs); } - static Aggregation AggUnique(boolean includeNulls, Object nonUniqueSentinel, String... pairs) { - return of(AggSpec.unique(includeNulls, nonUniqueSentinel), pairs); + static Aggregation AggUnique(boolean includeNulls, String... pairs) { + return AggUnique(includeNulls, Sentinel.of(), pairs); + } + + static Aggregation AggUnique(boolean includeNulls, Sentinel nonUniqueSentinel, String... pairs) { + return of(AggSpec.unique(includeNulls, nonUniqueSentinel.value()), pairs); } static Aggregation AggVar(String... pairs) { @@ -223,12 +228,18 @@ static PercentileOutput PctOut(double percentile, String outputColumn) { return PercentileOutput.of(percentile, outputColumn); } + static Sentinel Sentinel(Object value) { + return Sentinel.of(value); + } + + static Sentinel Sentinel() { + return Sentinel.of(); + } + V walk(V visitor); interface Visitor { - default void visit(Aggregations aggregations) { - aggregations.aggregations().forEach(a -> a.walk(this)); - } + void visit(Aggregations aggregations); void visit(ColumnAggregation columnAgg); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java index b5d6d91dcf3..fe0e73ec3d5 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationDescriptions.java @@ -25,6 +25,11 @@ Map getOut() { return out; } + @Override + public void visit(Aggregations aggregations) { + aggregations.aggregations().forEach(a -> a.walk(this)); + } + @Override public void visit(ColumnAggregation columnAgg) { visitColumnAgg(columnAgg.pair(), columnAgg.spec().description()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java index 35b5847e610..7f36f1ce066 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationOptimizer.java @@ -61,6 +61,11 @@ public List build() { return out; } + @Override + public void visit(Aggregations aggregations) { + aggregations.aggregations().forEach(a -> a.walk(this)); + } + @Override public void visit(ColumnAggregation columnAgg) { visitOrder.computeIfAbsent(columnAgg.spec(), k -> new ArrayList<>()).add(columnAgg.pair()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java index 49b6f89fa05..a1953122534 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java +++ b/table-api/src/main/java/io/deephaven/api/agg/AggregationPairs.java @@ -27,12 +27,17 @@ public static Stream outputsOf(Collection agg return of(aggregations).map(Pair::output); } - private Stream out; + protected Stream out; - Stream getOut() { + protected Stream getOut() { return Objects.requireNonNull(out); } + @Override + public void visit(Aggregations aggregations) { + out = aggregations.aggregations().stream().flatMap(AggregationPairs::of); + } + @Override public void visit(ColumnAggregation columnAgg) { out = Stream.of(columnAgg.pair()); diff --git a/table-api/src/main/java/io/deephaven/api/agg/util/Sentinel.java b/table-api/src/main/java/io/deephaven/api/agg/util/Sentinel.java new file mode 100644 index 00000000000..ba5a70660e4 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/util/Sentinel.java @@ -0,0 +1,32 @@ +package io.deephaven.api.agg.util; + +import io.deephaven.annotations.SimpleStyle; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +import javax.annotation.Nullable; + +/** + * Sentinel wrapper, used when specifying a weakly-typed sentinel value for certain aggregations. + */ +@Immutable +@SimpleStyle +public abstract class Sentinel { + + public static Sentinel of(Object value) { + return ImmutableSentinel.of(value); + } + + public static Sentinel of() { + return ImmutableSentinel.of(null); + } + + /** + * Get the sentinel value. + * + * @return The sentinel value + */ + @Parameter + @Nullable + public abstract Object value(); +} From 3468d7777df179fef22ddaad7349d5595fb699cf Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 31 Jan 2022 00:52:13 -0500 Subject: [PATCH 33/44] spotless + generate --- .../doc/io/deephaven/api/agg/Aggregation.json | 3 ++- .../api/agg/AggregationDescriptions.json | 2 +- .../api/agg/AggregationOptimizer.json | 2 +- .../io/deephaven/api/agg/AggregationPairs.json | 2 +- .../io/deephaven/api/agg/util/Sentinel.json | 10 ++++++++++ .../impl/by/rollup/RollupAggregationPairs.json | 11 +++++++++++ .../table/impl/by/AggregationProcessor.java | 18 ++++++++++-------- .../deephaven/engine/table/impl/TestAggBy.java | 3 +-- 8 files changed, 37 insertions(+), 14 deletions(-) create mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/util/Sentinel.json create mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.json diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json index 4f69f568778..5c0a3fa276b 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json @@ -22,11 +22,12 @@ "AggStd": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggSum": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggTDigest": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param compression: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggUnique": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param nonUniqueSentinel: java.lang.Object\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", + "AggUnique": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param includeNulls: boolean\n :param nonUniqueSentinel: io.deephaven.api.agg.util.Sentinel\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", "AggVar": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggWAvg": ":param weightColumn: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "AggWSum": ":param weightColumn: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", "PctOut": ":param percentile: double\n:param outputColumn: java.lang.String\n:return: io.deephaven.api.agg.util.PercentileOutput", + "Sentinel": "*Overload 1* \n :param value: java.lang.Object\n :return: io.deephaven.api.agg.util.Sentinel\n \n*Overload 2* \n :return: io.deephaven.api.agg.util.Sentinel", "of": "*Overload 1* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pair: java.lang.String\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param aggregations: io.deephaven.api.agg.Aggregation...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n Note: Java generics information - \n \n :param columnAggFactory: java.util.function.BiFunction\n :param inputColumn: java.lang.String\n :param inputs: INPUT_TYPE...\n :return: io.deephaven.api.agg.Aggregation", "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" }, diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json index 8ba6fd4b69e..6611c85a363 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json @@ -2,7 +2,7 @@ "className": "io.deephaven.api.agg.AggregationDescriptions", "methods": { "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.Map\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.Map", - "visit": "*Overload 1* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 3* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 4* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 5* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" }, "path": "io.deephaven.api.agg.AggregationDescriptions", "text": "A visitor to describe the input and aggregation column name pairs for aggregations.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json index a16fe3dd4b6..23f4ecccb03 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json @@ -3,7 +3,7 @@ "methods": { "build": ":return: java.util.List", "of": "Optimizes a collection of aggregations by grouping like-specced aggregations together. The\n input order will be preserved based on the spec-encountered order.\n\n:param aggregations: (java.util.Collection) - the aggregations\n:return: (java.util.List) the optimized aggregations", - "visit": "*Overload 1* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 3* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 4* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 5* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" }, "path": "io.deephaven.api.agg.AggregationOptimizer", "text": "Optimizes a collection of aggregations by grouping like-specced aggregations together.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json index 64d83365a40..29ce7efcb0c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json @@ -3,7 +3,7 @@ "methods": { "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", "outputsOf": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", - "visit": "*Overload 1* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 3* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 4* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 5* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" }, "path": "io.deephaven.api.agg.AggregationPairs", "text": "A visitor to get the ordered input/output column name pairs for aggregations.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/util/Sentinel.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/util/Sentinel.json new file mode 100644 index 00000000000..bfb126755ae --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/util/Sentinel.json @@ -0,0 +1,10 @@ +{ + "className": "io.deephaven.api.agg.util.Sentinel", + "methods": { + "of": "*Overload 1* \n :param value: java.lang.Object\n :return: io.deephaven.api.agg.util.Sentinel\n \n*Overload 2* \n :return: io.deephaven.api.agg.util.Sentinel", + "value": "Get the sentinel value.\n\n:return: (java.lang.Object) The sentinel value" + }, + "path": "io.deephaven.api.agg.util.Sentinel", + "text": "Sentinel wrapper, used when specifying a weakly-typed sentinel value for certain aggregations.", + "typeName": "class" +} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.json new file mode 100644 index 00000000000..d7c82b2722e --- /dev/null +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationPairs.json @@ -0,0 +1,11 @@ +{ + "className": "io.deephaven.engine.table.impl.by.rollup.RollupAggregationPairs", + "methods": { + "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", + "outputsOf": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", + "visit": "*Overload 1* \n :param nullColumns: io.deephaven.engine.table.impl.by.rollup.NullColumns\n \n*Overload 2* \n :param partition: io.deephaven.engine.table.impl.by.rollup.Partition" + }, + "path": "io.deephaven.engine.table.impl.by.rollup.RollupAggregationPairs", + "text": "A visitor to get the ordered input/output column name pairs for aggregations,\n including rollup aggregations.", + "typeName": "class" +} \ No newline at end of file diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 25376695e23..da531b51e90 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -156,10 +156,12 @@ public class AggregationProcessor implements AggregationContextFactory { private enum Type { + // @formatter:off NORMAL(false), ROLLUP_BASE(true), ROLLUP_REAGGREGATED(true), SELECT_DISTINCT(false); + // @formatter:on private final boolean isRollup; @@ -238,12 +240,13 @@ private AggregationProcessor( @NotNull final Type type) { this.aggregations = aggregations; this.type = type; - final String duplicationErrorMessage = - (type.isRollup - ? RollupAggregationPairs.outputsOf(aggregations) - : AggregationPairs.outputsOf(aggregations)) - .collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet().stream() - .filter(kv -> kv.getValue() > 1).map(kv -> kv.getKey() + " used " + kv.getValue() + " times") + final String duplicationErrorMessage = (type.isRollup + ? RollupAggregationPairs.outputsOf(aggregations) + : AggregationPairs.outputsOf(aggregations)) + .collect(Collectors.groupingBy(ColumnName::name, Collectors.counting())).entrySet() + .stream() + .filter(kv -> kv.getValue() > 1) + .map(kv -> kv.getKey() + " used " + kv.getValue() + " times") .collect(Collectors.joining(", ")); if (!duplicationErrorMessage.isBlank()) { throw new IllegalArgumentException("Duplicate output columns found: " + duplicationErrorMessage); @@ -1131,8 +1134,7 @@ private void reaggregateAsSum() { } private void reaggregateSsmBackedOperator( - TriFunction>, ColumnSource, String, - IterativeChunkedAggregationOperator> operatorFactory) { + TriFunction>, ColumnSource, String, IterativeChunkedAggregationOperator> operatorFactory) { for (final Pair pair : resultPairs) { final String resultName = pair.output().name(); final String ssmName = resultName + ROLLUP_DISTINCT_SSM_COLUMN_ID + ROLLUP_COLUMN_SUFFIX; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java index b638cb05e31..ee1aba5037e 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java @@ -636,8 +636,7 @@ public void testAggUniqueDefaultValues() { "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol"), "USym").sort("USym")); dataTable.aggBy(AggUnique(false, Sentinel(-2), "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", - "DoubleCol"), - "USym").sort("USym"); + "DoubleCol"), "USym").sort("USym"); // Byte out of range testUniqueOutOfRangeParams(Byte.class, dataTable, ((short) Byte.MIN_VALUE - 1), Byte.MIN_VALUE, From 4b85ee7391c99fc38ae2563dee3c5bf902d6e101 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 31 Jan 2022 09:09:52 -0500 Subject: [PATCH 34/44] Fix mod-detection off-by-one in TDigestPercentileOperator --- .../engine/table/impl/by/TDigestPercentileOperator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java index 9fce8fad764..60858f53efa 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/TDigestPercentileOperator.java @@ -189,7 +189,7 @@ public void propagateInitialState(@NotNull final QueryTable resultTable) { if (resultColumns.length == 0) { return; } - firstNewDestination = resultTable.getRowSet().lastRowKey(); + firstNewDestination = resultTable.getRowSet().lastRowKey() + 1; resultTable.getRowSet().forAllRowKeys(this::updateDestination); } @@ -210,7 +210,7 @@ public void propagateUpdates(@NotNull final TableUpdate downstream, @NotNull fin if (resultColumns.length == 0) { return; } - firstNewDestination = newDestinations.lastRowKey(); + firstNewDestination = newDestinations.lastRowKey() + 1; downstream.added().forAllRowKeys(this::updateDestination); if (modifiedThisStep) { downstream.modified().forAllRowKeys(this::updateDestination); From ba91e2cc6390c7fe36666d1115af5c7129f39fa3 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 31 Jan 2022 09:38:50 -0500 Subject: [PATCH 35/44] Clean up some AggregationFactory remnants in Python --- Generators/Generators.gradle | 2 - .../deephaven/AggregationFactory/__init__.py | 474 ------------------ Integrations/python/deephaven/__init__.py | 6 +- Integrations/python/test/testAggregation.py | 2 +- 4 files changed, 2 insertions(+), 482 deletions(-) delete mode 100644 Integrations/python/deephaven/AggregationFactory/__init__.py diff --git a/Generators/Generators.gradle b/Generators/Generators.gradle index 2ab7802250d..43f5ee40ec6 100644 --- a/Generators/Generators.gradle +++ b/Generators/Generators.gradle @@ -227,8 +227,6 @@ def pythonStaticMethodsArgs = ['io.deephaven.time.calendar.Calendars', 'io.deephaven.engine.util.TableTools', '/Generators/src/main/java/io/deephaven/pythonPreambles/TableToolsPreamble.txt', '/Integrations/python/deephaven/TableTools/__init__.py', - 'io.deephaven.engine.table.impl.by.AggregationFactory', - '/Integrations/python/deephaven/AggregationFactory/__init__.py', 'io.deephaven.plot.colors.ColorMaps', '/Integrations/python/deephaven/Plot/ColorMaps/__init__.py', 'io.deephaven.engine.util.WindowCheck', diff --git a/Integrations/python/deephaven/AggregationFactory/__init__.py b/Integrations/python/deephaven/AggregationFactory/__init__.py deleted file mode 100644 index 3471c49b646..00000000000 --- a/Integrations/python/deephaven/AggregationFactory/__init__.py +++ /dev/null @@ -1,474 +0,0 @@ - -""" -The AggregationFactory combines one or more aggregations into an AggregationSpec for use internally by the - implementation of Table.aggBy(io.deephaven.api.agg.Aggregation). - - - The intended use of this class is to call the AggCombo(AggregationElement...) method with a set of - aggregations defined by: - -* AggMin(java.lang.String...) -* AggMax(java.lang.String...) -* AggSum(java.lang.String...) -* AggAbsSum(java.lang.String...) -* AggVar(java.lang.String...) -* AggAvg(java.lang.String...) -* AggWAvg(java.lang.String, java.lang.String...) -* AggWSum(java.lang.String, java.lang.String...) -* AggMed(java.lang.String...) -* AggPct(double, java.lang.String...) -* AggStd(java.lang.String...) -* AggFirst(java.lang.String...) -* AggLast(java.lang.String...) -* AggCount(java.lang.String) -* AggCountDistinct(java.lang.String...) -* AggDistinct(java.lang.String...) -* AggGroup(java.lang.String...) -* AggSortedFirst(java.lang.String, java.lang.String...) -* AggSortedLast(java.lang.String, java.lang.String...) -""" - - -# -# Copyright (c) 2016-2021 Deephaven Data Labs and Patent Pending -# - -############################################################################## -# This code is auto generated. DO NOT EDIT FILE! -# Run "./gradlew :Generators:generatePythonIntegrationStaticMethods" to generate -############################################################################## - - -import jpy -import wrapt - - -_java_type_ = None # None until the first _defineSymbols() call - - -def _defineSymbols(): - """ - Defines appropriate java symbol, which requires that the jvm has been initialized through the :class:`jpy` module, - for use throughout the module AT RUNTIME. This is versus static definition upon first import, which would lead to an - exception if the jvm wasn't initialized BEFORE importing the module. - """ - - if not jpy.has_jvm(): - raise SystemError("No java functionality can be used until the JVM has been initialized through the jpy module") - - global _java_type_ - if _java_type_ is None: - # This will raise an exception if the desired object is not the classpath - _java_type_ = jpy.get_type("io.deephaven.engine.table.impl.by.AggregationFactory") - - -# every module method should be decorated with @_passThrough -@wrapt.decorator -def _passThrough(wrapped, instance, args, kwargs): - """ - For decoration of module methods, to define necessary symbols at runtime - - :param wrapped: the method to be decorated - :param instance: the object to which the wrapped function was bound when it was called - :param args: the argument list for `wrapped` - :param kwargs: the keyword argument dictionary for `wrapped` - :return: the decorated version of the method - """ - - _defineSymbols() - return wrapped(*args, **kwargs) - - -# Define all of our functionality, if currently possible -try: - _defineSymbols() -except Exception as e: - pass - - -@_passThrough -def Agg(*args): - """ - Create an aggregation. - - *Overload 1* - :param factory: (io.deephaven.engine.table.impl.by.AggregationSpec) - aggregation factory. - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 2* - :param factory: (io.deephaven.engine.table.impl.by.AggregationSpec) - aggregation factory. - :param matchPairs: (io.deephaven.engine.table.MatchPair...) - the columns to apply the aggregation to. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 3* - :param factoryType: (io.deephaven.engine.table.impl.by.AggType) - aggregation factory type. - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 4* - :param factoryType: (io.deephaven.engine.table.impl.by.AggType) - aggregation factory type. - :param matchPairs: (io.deephaven.engine.table.MatchPair...) - the columns to apply the aggregation to. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.Agg(*args) - - -@_passThrough -def AggAbsSum(*matchPairs): - """ - Create an absolute sum aggregation, equivalent to Table.absSumBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggAbsSum(*matchPairs) - - -@_passThrough -def AggAvg(*matchPairs): - """ - Create an average aggregation, equivalent to Table.avgBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggAvg(*matchPairs) - - -@_passThrough -def AggCombo(*aggregations): - """ - Create a new AggregationFactory suitable for passing to - QueryTable.by(AggregationSpec, io.deephaven.engine.table.impl.select.SelectColumn...). - - :param aggregations: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement...) - the aggregations to compute - :return: (io.deephaven.engine.table.impl.by.AggregationFactory) a new table with the specified aggregations. - """ - - return _java_type_.AggCombo(*aggregations) - - -@_passThrough -def AggCount(resultColumn): - """ - Create an count aggregation, equivalent to Table.countBy(String). - - :param resultColumn: (java.lang.String) - the name of the result column containing the count of each group - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggCount(resultColumn) - - -@_passThrough -def AggCountDistinct(*args): - """ - Create a distinct count aggregation. - - The output column contains the number of distinct values for the input column in that group. - - *Overload 1* - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...). Null values - are not counted. - - *Overload 2* - :param countNulls: (boolean) - if true null values are counted as a distinct value, otherwise null values are ignored - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggCountDistinct(*args) - - -@_passThrough -def AggDistinct(*args): - """ - Create a distinct aggregation. - - The output column contains a Vector with the distinct values for the input column within the group. - - *Overload 1* - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...). Null values - are ignored. - - *Overload 2* - :param countNulls: (boolean) - if true, then null values are included in the result, otherwise null values are ignored - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggDistinct(*args) - - -@_passThrough -def AggFirst(*matchPairs): - """ - Create a first aggregation, equivalent to Table.firstBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggFirst(*matchPairs) - - -@_passThrough -def AggFormula(formula, formulaParam, *matchPairs): - """ - Create a formula aggregation. - - :param formula: (java.lang.String) - the formula to apply to each group - :param formulaParam: (java.lang.String) - the parameter name within the formula - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggFormula(formula, formulaParam, *matchPairs) - - -@_passThrough -def AggGroup(*matchPairs): - """ - Create a group aggregation, equivalent to Table.groupBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggGroup(*matchPairs) - - -@_passThrough -def AggLast(*matchPairs): - """ - Create a last aggregation, equivalent to Table.lastBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggLast(*matchPairs) - - -@_passThrough -def AggMax(*matchPairs): - """ - Create a maximum aggregation, equivalent to Table.maxBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggMax(*matchPairs) - - -@_passThrough -def AggMed(*matchPairs): - """ - Create a median aggregation, equivalent to Table.medianBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggMed(*matchPairs) - - -@_passThrough -def AggMin(*matchPairs): - """ - Create a minimum aggregation, equivalent to Table.minBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggMin(*matchPairs) - - -@_passThrough -def AggPct(*args): - """ - Create a percentile aggregation. - - *Overload 1* - :param percentile: (double) - the percentile to calculate - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 2* - :param percentile: (double) - the percentile to calculate - :param averageMedian: (boolean) - if true, then when the upper values and lower values have an equal size; average the highest - lower value and lowest upper value to produce the median value for integers, longs, doubles, and floats - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggPct(*args) - - -@_passThrough -def AggSortedFirst(*args): - """ - Create a sorted first aggregation, equivalent to SortedBy.sortedFirstBy(io.deephaven.engine.table.Table, java.lang.String). - - *Overload 1* - :param sortColumn: (java.lang.String) - the column to sort by - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 2* - :param sortColumns: (java.lang.String[]) - the column to sort by - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggSortedFirst(*args) - - -@_passThrough -def AggSortedLast(*args): - """ - Create a sorted last aggregation, equivalent to SortedBy.sortedLastBy(io.deephaven.engine.table.Table, java.lang.String). - - *Overload 1* - :param sortColumn: (java.lang.String) - the column to sort by - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 2* - :param sortColumns: (java.lang.String[]) - the columns to sort by - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggSortedLast(*args) - - -@_passThrough -def AggStd(*matchPairs): - """ - Create a standard deviation aggregation, equivalent to Table.stdBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggStd(*matchPairs) - - -@_passThrough -def AggSum(*matchPairs): - """ - Create a summation aggregation, equivalent to Table.sumBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggSum(*matchPairs) - - -@_passThrough -def AggUnique(*args): - """ - Create a Unique aggregation. - - The output column contains a value of the same type as the input column which contains - - * The "no key value" - if there are no values present - * The single unique value - if there is only a single value present - * The "non unique value" - if there are more than 1 distinct values present - - *Overload 1* - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - - *Overload 2* - :param countNulls: (boolean) - if true, then null values are included in the result, otherwise null values are ignored - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...). Output - columns contain null if there are no values present or there are more than 1 distinct values present. - - *Overload 3* - :param countNulls: (boolean) - if true, then null values are included in the result, otherwise null values are ignored - :param noKeyValue: (java.lang.Object) - the value to use if there are no values present - :param nonUniqueValue: (java.lang.Object) - the value to use if there are more than 1 values present - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggUnique(*args) - - -@_passThrough -def AggVar(*matchPairs): - """ - Create a variance aggregation, equivalent to Table.varBy(String...). - - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggVar(*matchPairs) - - -@_passThrough -def AggWAvg(weight, *matchPairs): - """ - Create a weighted average aggregation, equivalent to Table.wavgBy(String, String...). - - :param weight: (java.lang.String) - the name of the column to use as the weight for the average - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggWAvg(weight, *matchPairs) - - -@_passThrough -def AggWSum(weight, *matchPairs): - """ - Create a weighted sum aggregation, equivalent to Table.wsumBy(String, String...). - - :param weight: (java.lang.String) - the name of the column to use as the weight for the sum - :param matchPairs: (java.lang.String...) - the columns to apply the aggregation to in the form Output=Input, if the Output and Input have - the same name, then the column name can be specified. - :return: (io.deephaven.engine.table.impl.by.AggregationFactory.AggregationElement) a AggregationElement object suitable for passing to AggCombo(AggregationElement...) - """ - - return _java_type_.AggWSum(weight, *matchPairs) diff --git a/Integrations/python/deephaven/__init__.py b/Integrations/python/deephaven/__init__.py index 283966dde42..97aafc2e538 100644 --- a/Integrations/python/deephaven/__init__.py +++ b/Integrations/python/deephaven/__init__.py @@ -9,8 +9,6 @@ * Calendars imported as cals -* AggregationFactory imported as af - * DateTimeUtils imported as dtu * MovingAverages imported as mavg @@ -58,7 +56,7 @@ 'Aggregation', 'ColumnName', 'ColumnRenderersBuilder', 'DistinctFormatter', 'DownsampledWhereFilter', 'DynamicTableWriter', 'LayoutHintBuilder', 'Replayer', 'SmartKey', 'SortColumn', 'TotalsTableBuilder', 'WindowCheck', # from TableManipulation - "cals", "af", "dtu", "figw", "mavg", "plt", "pt", "ttools", "tloggers", # subpackages with abbreviated names + "cals", "dtu", "figw", "mavg", "plt", "pt", "ttools", "tloggers", # subpackages with abbreviated names "read_csv", "write_csv" # from csv ] @@ -81,7 +79,6 @@ from . import Calendars as cals, \ - AggregationFactory as af, \ DateTimeUtils as dtu, \ MovingAverages as mavg, \ ConsumeKafka as ck, \ @@ -107,7 +104,6 @@ def initialize(): # ensure that all the symbols are called and reimport the broken symbols cals._defineSymbols() - af._defineSymbols() dtu._defineSymbols() dh._defineSymbols() ck._defineSymbols() diff --git a/Integrations/python/test/testAggregation.py b/Integrations/python/test/testAggregation.py index 9a9ed1b4014..d0879da0ce4 100644 --- a/Integrations/python/test/testAggregation.py +++ b/Integrations/python/test/testAggregation.py @@ -23,7 +23,7 @@ class TestAggregation(unittest.TestCase): """ - Test cases for the deephaven.AggregationFactory module (performed locally) - + Test cases for the deephaven.Aggregation module (performed locally) - """ def testAggMethods(self): From 6adf0c779cc71aaa6d63abc071ff6340055c940e Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Mon, 31 Jan 2022 11:45:46 -0500 Subject: [PATCH 36/44] Re-run json doc generation. --- .../io/deephaven/engine/exceptions/TableMapException.json | 7 ------- .../deephaven/doc/io/deephaven/kafka/KafkaTools.json | 5 ++++- 2 files changed, 4 insertions(+), 8 deletions(-) delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/engine/exceptions/TableMapException.json diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/exceptions/TableMapException.json b/Integrations/python/deephaven/doc/io/deephaven/engine/exceptions/TableMapException.json deleted file mode 100644 index 53a139b6c5e..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/exceptions/TableMapException.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "className": "io.deephaven.engine.exceptions.TableMapException", - "methods": {}, - "path": "io.deephaven.engine.exceptions.TableMapException", - "text": "An exception denoting a problem with a io.deephaven.engine.table.impl.TableMap operation", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools.json b/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools.json index 81ad5ea5c6b..56bd9a5ce9f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools.json +++ b/Integrations/python/deephaven/doc/io/deephaven/kafka/KafkaTools.json @@ -2,12 +2,15 @@ "className": "io.deephaven.kafka.KafkaTools", "methods": { "avroSchemaToColumnDefinitions": "**Incompatible overloads text - text from the first overload:**\n\nConvert an Avro schema to a list of column definitions, mapping every avro field to a column of the same name.\n\n*Overload 1* \n :param columnsOut: java.util.List>\n :param fieldPathToColumnNameOut: java.util.Map\n :param schema: org.apache.avro.Schema\n :param requestedFieldPathToColumnName: java.util.function.Function\n \n*Overload 2* \n :param columnsOut: (java.util.List>) - Column definitions for output; should be empty on entry.\n :param schema: (org.apache.avro.Schema) - Avro schema\n :param requestedFieldPathToColumnName: (java.util.function.Function) - An optional mapping to specify selection and naming of columns from Avro\n fields, or null for map all fields using field path for column name.\n \n*Overload 3* \n :param columnsOut: (java.util.List>) - Column definitions for output; should be empty on entry.\n :param schema: (org.apache.avro.Schema) - Avro schema", + "columnDefinitionsToAvroSchema": ":param t: io.deephaven.engine.table.Table\n:param schemaName: java.lang.String\n:param namespace: java.lang.String\n:param colProps: java.util.Properties\n:param includeOnly: java.util.function.Predicate\n:param exclude: java.util.function.Predicate\n:param colPropsOut: org.apache.commons.lang3.mutable.MutableObject\n:return: org.apache.avro.Schema", "consumeToTable": "Consume from Kafka to a Deephaven table.\n\n:param kafkaProperties: (java.util.Properties) - Properties to configure this table and also to be passed to create the KafkaConsumer\n:param topic: (java.lang.String) - Kafka topic name\n:param partitionFilter: (java.util.function.IntPredicate) - A predicate returning true for the partitions to consume. The convenience constant\n ALL_PARTITIONS is defined to facilitate requesting all partitions.\n:param partitionToInitialOffset: (java.util.function.IntToLongFunction) - A function specifying the desired initial offset for each partition consumed\n:param keySpec: (io.deephaven.kafka.KafkaTools.Consume.KeyOrValueSpec) - Conversion specification for Kafka record keys\n:param valueSpec: (io.deephaven.kafka.KafkaTools.Consume.KeyOrValueSpec) - Conversion specification for Kafka record values\n:param resultType: (io.deephaven.kafka.KafkaTools.TableType) - KafkaTools.TableType specifying the type of the expected result\n:return: (io.deephaven.engine.table.Table) The result table containing Kafka stream data formatted according to resultType", "friendlyNameToTableType": "Map \"Python-friendly\" table type name to a KafkaTools.TableType.\n\n:param typeName: (java.lang.String) - The friendly name\n:return: (io.deephaven.kafka.KafkaTools.TableType) The mapped KafkaTools.TableType", "getAvroSchema": "**Incompatible overloads text - text from the first overload:**\n\nFetch an Avro schema from a Confluent compatible Schema Server.\n\n*Overload 1* \n :param schemaServerUrl: (java.lang.String) - The schema server URL\n :param resourceName: (java.lang.String) - The resource name that the schema is known as in the schema server\n :param version: (java.lang.String) - The version to fetch, or the string \"latest\" for the latest version.\n :return: (org.apache.avro.Schema) An Avro schema.\n \n*Overload 2* \n :param schemaServerUrl: (java.lang.String) - The schema server URL\n :param resourceName: (java.lang.String) - The resource name that the schema is known as in the schema server\n :return: (org.apache.avro.Schema) An Avro schema.", "partitionFilterFromArray": ":param partitions: int[]\n:return: java.util.function.IntPredicate", "partitionToOffsetFromParallelArrays": ":param partitions: int[]\n:param offsets: long[]\n:return: java.util.function.IntToLongFunction", - "produceFromTable": "Consume from Kafka to a Deephaven table.\n\n:param table: (io.deephaven.engine.table.Table) - The table used as a source of data to be sent to Kafka.\n:param kafkaProperties: (java.util.Properties) - Properties to be passed to create the associated KafkaProducer.\n:param topic: (java.lang.String) - Kafka topic name\n:param keySpec: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) - Conversion specification for Kafka record keys from table column data.\n:param valueSpec: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) - Conversion specification for Kafka record values from table column data.\n:param lastByKeyColumns: (boolean) - Whether to publish only the last record for each unique key. Ignored when keySpec\n is IGNORE. If keySpec != null && !lastByKeyColumns, it is expected that table will\n not produce any row shifts; that is, the publisher expects keyed tables to be streams, add-only, or\n aggregated.\n:return: (java.lang.Runnable) a callback to stop producing and shut down the associated table listener; note a caller should keep a\n reference to this return value to ensure liveliness." + "predicateFromSet": ":param set: java.util.Set\n:return: java.util.function.Predicate", + "produceFromTable": "Consume from Kafka to a Deephaven table.\n\n:param table: (io.deephaven.engine.table.Table) - The table used as a source of data to be sent to Kafka.\n:param kafkaProperties: (java.util.Properties) - Properties to be passed to create the associated KafkaProducer.\n:param topic: (java.lang.String) - Kafka topic name\n:param keySpec: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) - Conversion specification for Kafka record keys from table column data.\n:param valueSpec: (io.deephaven.kafka.KafkaTools.Produce.KeyOrValueSpec) - Conversion specification for Kafka record values from table column data.\n:param lastByKeyColumns: (boolean) - Whether to publish only the last record for each unique key. Ignored when keySpec\n is IGNORE. If keySpec != null && !lastByKeyColumns, it is expected that table will\n not produce any row shifts; that is, the publisher expects keyed tables to be streams, add-only, or\n aggregated.\n:return: (java.lang.Runnable) a callback to stop producing and shut down the associated table listener; note a caller should keep a\n reference to this return value to ensure liveliness.", + "putAvroSchema": "Push an Avro schema from a Confluent compatible Schema Server.\n\n:param schema: (org.apache.avro.Schema) - An Avro schema\n:param schemaServerUrl: (java.lang.String) - The schema server URL\n:param resourceName: (java.lang.String) - The resource name that the schema will be known as in the schema server\n:return: (java.lang.String) The version for the added resource as returned by schema server." }, "path": "io.deephaven.kafka.KafkaTools", "typeName": "class" From d8cea5948594d1300fa0afc9eac6c6a15cdb8381 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 1 Feb 2022 11:52:22 -0500 Subject: [PATCH 37/44] Re-order AggSpecs in Visitor to alphabetize them correctly, and add a missing AggFreeze factory method. --- .../table/impl/AggAllByCopyAttributes.java | 18 +++---- .../engine/table/impl/AggAllByUseTable.java | 18 +++---- .../table/impl/by/AggregationProcessor.java | 52 +++++++++---------- .../client/impl/BatchTableRequestBuilder.java | 28 +++++----- .../qst/table/AggAllByExclusions.java | 14 ++--- .../io/deephaven/api/agg/Aggregation.java | 5 ++ .../deephaven/api/agg/AggregationOutputs.java | 14 ----- .../io/deephaven/api/agg/spec/AggSpec.java | 12 ++--- .../api/agg/spec/AggSpecColumnReferences.java | 14 ++--- 9 files changed, 83 insertions(+), 92 deletions(-) delete mode 100644 table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java index 6144010ceea..95406168673 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByCopyAttributes.java @@ -23,19 +23,13 @@ public void visit(AggSpecAbsSum absSum) {} public void visit(AggSpecApproximatePercentile approxPct) {} @Override - public void visit(AggSpecCountDistinct countDistinct) {} - - @Override - public void visit(AggSpecDistinct distinct) {} - - @Override - public void visit(AggSpecFreeze freeze) {} + public void visit(AggSpecAvg avg) {} @Override - public void visit(AggSpecGroup group) {} + public void visit(AggSpecCountDistinct countDistinct) {} @Override - public void visit(AggSpecAvg avg) {} + public void visit(AggSpecDistinct distinct) {} @Override public void visit(AggSpecFirst first) { @@ -45,6 +39,12 @@ public void visit(AggSpecFirst first) { @Override public void visit(AggSpecFormula formula) {} + @Override + public void visit(AggSpecFreeze freeze) {} + + @Override + public void visit(AggSpecGroup group) {} + @Override public void visit(AggSpecLast last) { parent.copyAttributes(result, CopyAttributeOperation.LastBy); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java index 70bc8b18727..b4bc5a1145c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/AggAllByUseTable.java @@ -41,37 +41,37 @@ public void visit(AggSpecApproximatePercentile approxPct) { } @Override - public void visit(AggSpecCountDistinct countDistinct) { + public void visit(AggSpecAvg avg) { drop(); } @Override - public void visit(AggSpecDistinct distinct) { - keep(); + public void visit(AggSpecCountDistinct countDistinct) { + drop(); } @Override - public void visit(AggSpecFreeze freeze) { + public void visit(AggSpecDistinct distinct) { keep(); } @Override - public void visit(AggSpecGroup group) { - drop(); + public void visit(AggSpecFirst first) { + keep(); } @Override - public void visit(AggSpecAvg avg) { + public void visit(AggSpecFormula formula) { drop(); } @Override - public void visit(AggSpecFirst first) { + public void visit(AggSpecFreeze freeze) { keep(); } @Override - public void visit(AggSpecFormula formula) { + public void visit(AggSpecGroup group) { drop(); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index da531b51e90..e5af7f621b4 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -660,6 +660,11 @@ public void visit(@NotNull final AggSpecApproximatePercentile approxPct) { addApproximatePercentileOperators(approxPct.percentile(), approxPct.compression()); } + @Override + public void visit(@NotNull final AggSpecAvg avg) { + addBasicOperators((t, n) -> makeAvgOperator(t, n, false)); + } + @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { addBasicOperators((t, n) -> makeCountDistinctOperator(t, n, countDistinct.countNulls(), false, false)); @@ -670,22 +675,6 @@ public void visit(@NotNull final AggSpecDistinct distinct) { addBasicOperators((t, n) -> makeDistinctOperator(t, n, distinct.includeNulls(), false, false)); } - @Override - public void visit(AggSpecFreeze freeze) { - addFreezeOperators(); - } - - @Override - public void visit(@NotNull final AggSpecGroup group) { - streamUnsupported("Group"); - addNoInputOperator(new GroupByChunkedOperator(table, true, MatchPair.fromPairs(resultPairs))); - } - - @Override - public void visit(@NotNull final AggSpecAvg avg) { - addBasicOperators((t, n) -> makeAvgOperator(t, n, false)); - } - @Override public void visit(@NotNull final AggSpecFirst first) { addFirstOrLastOperators(true, null); @@ -701,6 +690,17 @@ public void visit(@NotNull final AggSpecFormula formula) { addNoInputOperator(formulaChunkedOperator); } + @Override + public void visit(AggSpecFreeze freeze) { + addFreezeOperators(); + } + + @Override + public void visit(@NotNull final AggSpecGroup group) { + streamUnsupported("Group"); + addNoInputOperator(new GroupByChunkedOperator(table, true, MatchPair.fromPairs(resultPairs))); + } + @Override public void visit(@NotNull final AggSpecLast last) { addFirstOrLastOperators(false, null); @@ -920,6 +920,11 @@ public void visit(@NotNull final AggSpecAbsSum absSum) { addBasicOperators((t, n) -> makeSumOperator(t, n, true)); } + @Override + public void visit(@NotNull final AggSpecAvg avg) { + addBasicOperators((t, n) -> makeAvgOperator(t, n, true)); + } + @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { addBasicOperators((t, n) -> makeCountDistinctOperator(t, n, countDistinct.countNulls(), true, false)); @@ -930,11 +935,6 @@ public void visit(@NotNull final AggSpecDistinct distinct) { addBasicOperators((t, n) -> makeDistinctOperator(t, n, distinct.includeNulls(), true, false)); } - @Override - public void visit(@NotNull final AggSpecAvg avg) { - addBasicOperators((t, n) -> makeAvgOperator(t, n, true)); - } - @Override public void visit(@NotNull final AggSpecFirst first) { addFirstOrLastOperators(true, makeRedirectionName(nextColumnIdentifier++)); @@ -1051,6 +1051,11 @@ public void visit(@NotNull final AggSpecAbsSum absSum) { reaggregateAsSum(); } + @Override + public void visit(@NotNull final AggSpecAvg avg) { + reaggregateAvgOperator(); + } + @Override public void visit(@NotNull final AggSpecCountDistinct countDistinct) { reaggregateSsmBackedOperator((ssmSrc, priorResultSrc, n) -> makeCountDistinctOperator( @@ -1063,11 +1068,6 @@ public void visit(@NotNull final AggSpecDistinct distinct) { priorResultSrc.getComponentType(), n, distinct.includeNulls(), true, true)); } - @Override - public void visit(@NotNull final AggSpecAvg avg) { - reaggregateAvgOperator(); - } - @Override public void visit(@NotNull final AggSpecFirst first) { reaggregateFirstOrLastOperator(true); diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index 6287c4b31fd..ca4bb92dfb0 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -692,42 +692,42 @@ public void visit(AggSpecApproximatePercentile approxPct) { } @Override - public void visit(AggSpecCountDistinct countDistinct) { - throw new UnsupportedOperationException( - "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + public void visit(AggSpecAvg avg) { + out = of(AggType.AVG, pairs).build(); } @Override - public void visit(AggSpecDistinct distinct) { + public void visit(AggSpecCountDistinct countDistinct) { throw new UnsupportedOperationException( "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } @Override - public void visit(AggSpecFreeze freeze) { + public void visit(AggSpecDistinct distinct) { throw new UnsupportedOperationException( "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } @Override - public void visit(AggSpecGroup group) { - out = of(AggType.GROUP, pairs).build(); + public void visit(AggSpecFirst first) { + out = of(AggType.FIRST, pairs).build(); } @Override - public void visit(AggSpecAvg avg) { - out = of(AggType.AVG, pairs).build(); + public void visit(AggSpecFormula formula) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } @Override - public void visit(AggSpecFirst first) { - out = of(AggType.FIRST, pairs).build(); + public void visit(AggSpecFreeze freeze) { + throw new UnsupportedOperationException( + "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } @Override - public void visit(AggSpecFormula formula) { - throw new UnsupportedOperationException( - "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); + public void visit(AggSpecGroup group) { + out = of(AggType.GROUP, pairs).build(); } @Override diff --git a/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java b/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java index e9996fea9a2..c3c05ecdfa4 100644 --- a/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java +++ b/qst/src/main/java/io/deephaven/qst/table/AggAllByExclusions.java @@ -41,37 +41,37 @@ public void visit(AggSpecApproximatePercentile approxPct) { } @Override - public void visit(AggSpecCountDistinct countDistinct) { + public void visit(AggSpecAvg avg) { out = Collections.emptySet(); } @Override - public void visit(AggSpecDistinct distinct) { + public void visit(AggSpecCountDistinct countDistinct) { out = Collections.emptySet(); } @Override - public void visit(AggSpecFreeze freeze) { + public void visit(AggSpecDistinct distinct) { out = Collections.emptySet(); } @Override - public void visit(AggSpecGroup group) { + public void visit(AggSpecFirst first) { out = Collections.emptySet(); } @Override - public void visit(AggSpecAvg avg) { + public void visit(AggSpecFormula formula) { out = Collections.emptySet(); } @Override - public void visit(AggSpecFirst first) { + public void visit(AggSpecFreeze freeze) { out = Collections.emptySet(); } @Override - public void visit(AggSpecFormula formula) { + public void visit(AggSpecGroup group) { out = Collections.emptySet(); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index d579a9c6e0f..95f72766403 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -2,6 +2,7 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.api.agg.spec.AggSpecFreeze; import io.deephaven.api.agg.util.PercentileOutput; import io.deephaven.api.agg.util.Sentinel; @@ -120,6 +121,10 @@ static Aggregation AggFormula(String formula, String formulaParam, String... pai return of(AggSpec.formula(formula, formulaParam), pairs); } + static Aggregation AggFreeze(String... pairs) { + return of(AggSpec.freeze(), pairs); + } + static Aggregation AggGroup(String... pairs) { return of(AggSpec.group(), pairs); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java b/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java deleted file mode 100644 index c098a0d5926..00000000000 --- a/table-api/src/main/java/io/deephaven/api/agg/AggregationOutputs.java +++ /dev/null @@ -1,14 +0,0 @@ -package io.deephaven.api.agg; - -import io.deephaven.api.ColumnName; - -import java.util.Collection; -import java.util.Objects; -import java.util.stream.Stream; - -/** - * Helper to get the ordered output {@link ColumnName column names} for {@link Aggregation aggregations}. - */ -public class AggregationOutputs { - -} diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 2dd1b65dbc9..22643d255c0 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -169,20 +169,20 @@ interface Visitor { void visit(AggSpecApproximatePercentile approxPct); + void visit(AggSpecAvg avg); + void visit(AggSpecCountDistinct countDistinct); void visit(AggSpecDistinct distinct); - void visit(AggSpecFreeze freeze); - - void visit(AggSpecGroup group); - - void visit(AggSpecAvg avg); - void visit(AggSpecFirst first); void visit(AggSpecFormula formula); + void visit(AggSpecFreeze freeze); + + void visit(AggSpecGroup group); + void visit(AggSpecLast last); void visit(AggSpecMax max); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java index aa8b96f31f5..d3ea6727da8 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecColumnReferences.java @@ -31,37 +31,37 @@ public void visit(AggSpecApproximatePercentile approxPct) { } @Override - public void visit(AggSpecCountDistinct countDistinct) { + public void visit(AggSpecAvg avg) { out = Collections.emptySet(); } @Override - public void visit(AggSpecDistinct distinct) { + public void visit(AggSpecCountDistinct countDistinct) { out = Collections.emptySet(); } @Override - public void visit(AggSpecFreeze freeze) { + public void visit(AggSpecDistinct distinct) { out = Collections.emptySet(); } @Override - public void visit(AggSpecGroup group) { + public void visit(AggSpecFirst first) { out = Collections.emptySet(); } @Override - public void visit(AggSpecAvg avg) { + public void visit(AggSpecFormula formula) { out = Collections.emptySet(); } @Override - public void visit(AggSpecFirst first) { + public void visit(AggSpecFreeze freeze) { out = Collections.emptySet(); } @Override - public void visit(AggSpecFormula formula) { + public void visit(AggSpecGroup group) { out = Collections.emptySet(); } From a5230bd74862e7d5fd279afb5c72a94449bf6c4e Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 1 Feb 2022 21:33:21 -0500 Subject: [PATCH 38/44] Re-write numeric compatibility testing and conversion code for AggUnique in AggregationProcessor --- .../table/impl/by/AggregationProcessor.java | 207 +++++++++++++++--- .../engine/table/impl/TestAggBy.java | 27 ++- 2 files changed, 200 insertions(+), 34 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index e5af7f621b4..7a19cccded8 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -138,13 +138,16 @@ import static io.deephaven.engine.table.impl.RollupAttributeCopier.LEAF_WITHCONSTITUENTS_INSTANCE; import static io.deephaven.engine.table.impl.RollupInfo.ROLLUP_COLUMN; import static io.deephaven.engine.table.impl.by.IterativeChunkedAggregationOperator.ZERO_LENGTH_ITERATIVE_CHUNKED_AGGREGATION_OPERATOR_ARRAY; -import static io.deephaven.engine.table.impl.by.RollupConstants.*; -import static io.deephaven.util.QueryConstants.NULL_BYTE; -import static io.deephaven.util.QueryConstants.NULL_DOUBLE; -import static io.deephaven.util.QueryConstants.NULL_FLOAT; -import static io.deephaven.util.QueryConstants.NULL_INT; -import static io.deephaven.util.QueryConstants.NULL_LONG; -import static io.deephaven.util.QueryConstants.NULL_SHORT; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_COLUMN_SUFFIX; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_DISTINCT_SSM_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_NAN_COUNT_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_NI_COUNT_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_NONNULL_COUNT_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_PI_COUNT_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_RUNNING_SUM2_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROLLUP_RUNNING_SUM_COLUMN_ID; +import static io.deephaven.engine.table.impl.by.RollupConstants.ROW_REDIRECTION_PREFIX; +import static io.deephaven.util.QueryConstants.*; import static io.deephaven.util.type.TypeUtils.getBoxedType; import static io.deephaven.util.type.TypeUtils.isNumeric; import static io.deephaven.util.type.TypeUtils.unbox; @@ -1554,13 +1557,12 @@ private static IterativeChunkedAggregationOperator makeUniqueOperator( return reaggregated ? new ShortRollupUniqueOperator(resultName, includeNulls, onsAsType, nusAsType) : new ShortChunkedUniqueOperator(resultName, includeNulls, exposeInternal, onsAsType, nusAsType); - } else { - return reaggregated - ? new ObjectRollupUniqueOperator(type, resultName, includeNulls, onlyNullsSentinel, - nonUniqueSentinel) - : new ObjectChunkedUniqueOperator(type, resultName, includeNulls, exposeInternal, onlyNullsSentinel, - nonUniqueSentinel); } + final Object onsAsType = maybeConvertType(type, onlyNullsSentinel); + final Object nusAsType = maybeConvertType(type, nonUniqueSentinel); + return reaggregated + ? new ObjectRollupUniqueOperator(type, resultName, includeNulls, onsAsType, nusAsType) + : new ObjectChunkedUniqueOperator(type, resultName, includeNulls, exposeInternal, onsAsType, nusAsType); } private static void checkType(@NotNull final String name, @NotNull final String valueIntent, @@ -1568,11 +1570,11 @@ private static void checkType(@NotNull final String name, @NotNull final String expected = getBoxedType(expected); if (value != null && !expected.isAssignableFrom(value.getClass())) { if (isNumeric(expected) && isNumeric(value.getClass())) { - if (checkNumericCompatibility((Number) value, expected)) { + if (isNumericallyCompatible((Number) value, expected)) { return; } throw new IllegalArgumentException( - String.format("For result column %s the %s %s is larger than can be represented with a %s", + String.format("For result column %s the %s %s is out of range for %s", name, valueIntent, value, expected.getName())); } throw new IllegalArgumentException( @@ -1581,23 +1583,166 @@ private static void checkType(@NotNull final String name, @NotNull final String } } - private static boolean checkNumericCompatibility(@NotNull final Number value, @NotNull final Class expected) { - if (expected == Byte.class) { - return Byte.MIN_VALUE <= value.longValue() && value.longValue() <= Byte.MAX_VALUE; - } else if (expected == Short.class) { - return Short.MIN_VALUE <= value.longValue() && value.longValue() <= Short.MAX_VALUE; - } else if (expected == Integer.class) { - return Integer.MIN_VALUE <= value.longValue() && value.longValue() <= Integer.MAX_VALUE; - } else if (expected == Long.class) { - return new BigInteger(value.toString()).compareTo(BigInteger.valueOf(Long.MIN_VALUE)) >= 0 && - new BigInteger(value.toString()).compareTo(BigInteger.valueOf(Long.MAX_VALUE)) <= 0; - } else if (expected == Float.class) { - return value.getClass() != Double.class; - } else if (expected == Double.class) { - return value.getClass() != BigDecimal.class; - } else { - return expected == BigDecimal.class || expected == BigInteger.class; + private static Object maybeConvertType(@NotNull Class expected, final Object value) { + // We expect that checkType was already called and didn't throw... + if (value == null) { + return null; + } + if (expected.isAssignableFrom(value.getClass())) { + return value; + } + if (expected == BigInteger.class) { + return NumericConverter.lookup(value.getClass()).toBigInteger((Number) value); + } + return NumericConverter.lookup(value.getClass()).toBigDecimal((Number) value); + } + + private interface NumericConverter { + BigInteger toBigInteger(@Nullable final Number value); + BigDecimal toBigDecimal(@Nullable final Number value); + + private static NumericConverter lookup(@NotNull final Class numberClass) { + final IntegralType integralType = IntegralType.lookup(numberClass); + if (integralType != null) { + return integralType; + } + return FloatingPointType.lookup(numberClass); + } + } + + private enum IntegralType implements NumericConverter { + // @formatter:off + BYTE (n -> BigInteger.valueOf(n.byteValue()), MIN_BYTE, MAX_BYTE ), + SHORT (n -> BigInteger.valueOf(n.shortValue()), MIN_SHORT, MAX_SHORT), + INTEGER (n -> BigInteger.valueOf(n.intValue()), MIN_INT, MAX_INT ), + LONG (n -> BigInteger.valueOf(n.longValue()), MIN_LONG, MAX_LONG ), + BIGINTEGER(n -> (BigInteger) n, null, null ); + // @formatter:on + + private final Function toBigInteger; + private final BigInteger lowerBound; + private final BigInteger upperBound; + + IntegralType(@NotNull final Function toBigInteger, + @Nullable final Number lowerBound, + @Nullable final Number upperBound) { + this.toBigInteger = toBigInteger; + this.lowerBound = toBigInteger(lowerBound); + this.upperBound = toBigInteger(upperBound); + } + + @Override + public BigInteger toBigInteger(@Nullable final Number value) { + return value == null ? null : toBigInteger.apply(value); + } + + @Override + public BigDecimal toBigDecimal(@Nullable final Number value) { + return value == null ? null : new BigDecimal(toBigInteger.apply(value)); } + + private boolean inRange(@Nullable final BigInteger value) { + if (value == null) { + return true; + } + return (lowerBound == null || lowerBound.compareTo(value) <= 0) && + (upperBound == null || upperBound.compareTo(value) >= 0); + } + + private static IntegralType lookup(@NotNull final Class numberClass) { + try { + return valueOf(numberClass.getSimpleName().toUpperCase()); + } catch (IllegalArgumentException e) { + return null; + } + } + } + + private enum FloatingPointType implements NumericConverter { + // @formatter:off + FLOAT( n -> BigDecimal.valueOf(n.floatValue()), MIN_FINITE_FLOAT, MAX_FINITE_FLOAT ), + DOUBLE( n -> BigDecimal.valueOf(n.doubleValue()), MIN_FINITE_DOUBLE, MAX_FINITE_DOUBLE), + BIGDECIMAL(n -> (BigDecimal) n, null, null ); + // @formatter:on + + private final Function toBigDecimal; + private final BigDecimal lowerBound; + private final BigDecimal upperBound; + + FloatingPointType(@NotNull final Function toBigDecimal, + @Nullable final Number lowerBound, + @Nullable final Number upperBound) { + this.toBigDecimal = toBigDecimal; + this.lowerBound = toBigDecimal(lowerBound); + this.upperBound = toBigDecimal(upperBound); + } + + @Override + public BigInteger toBigInteger(@Nullable final Number value) { + return value == null ? null : toBigDecimal.apply(value).toBigIntegerExact(); + } + + @Override + public BigDecimal toBigDecimal(@Nullable final Number value) { + return value == null ? null : toBigDecimal.apply(value); + } + + private boolean inRange(@Nullable final BigDecimal value) { + if (value == null) { + return true; + } + return (lowerBound == null || lowerBound.compareTo(value) <= 0) && + (upperBound == null || upperBound.compareTo(value) >= 0); + } + + private static FloatingPointType lookup(@NotNull final Class numberClass) { + try { + return valueOf(numberClass.getSimpleName().toUpperCase()); + } catch (IllegalArgumentException e) { + return null; + } + } + } + + private static boolean isNumericallyCompatible(@NotNull final Number value, + @NotNull final Class expected) { + final NumericConverter valueConverter = NumericConverter.lookup(value.getClass()); + if (valueConverter == null) { + // value is not a recognized type + return false; + } + + final IntegralType expectedIntegralType = IntegralType.lookup(expected); + if (expectedIntegralType != null) { + // expected is a recognized integral type, just check range as a big int + try { + return expectedIntegralType.inRange(valueConverter.toBigInteger(value)); + } catch (ArithmeticException e) { + // value is a floating point number with a fractional part + return false; + } + } + + final FloatingPointType expectedFloatingPointType = FloatingPointType.lookup(expected); + if (expectedFloatingPointType == null) { + // expected is not a recognized type + return false; + } + + // check range as a big decimal + if (expectedFloatingPointType.inRange(valueConverter.toBigDecimal(value))) { + return true; + } + + // value might be out of range, or might not be finite... + if (expectedFloatingPointType == FloatingPointType.BIGDECIMAL || + (valueConverter != FloatingPointType.FLOAT && valueConverter != FloatingPointType.DOUBLE)) { + // no way to represent NaN or infinity, so value is just out of range + return false; + } + + // if we're not finite, we can cast to a float or double successfully + return !Double.isFinite(value.doubleValue()); } private static IterativeChunkedAggregationOperator makeAvgOperator( diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java index ee1aba5037e..5a9d64546a9 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/TestAggBy.java @@ -20,6 +20,7 @@ import io.deephaven.engine.table.impl.util.ColumnHolder; import java.lang.reflect.Array; +import java.math.BigDecimal; import java.math.BigInteger; import java.util.Arrays; import java.util.Collection; @@ -627,16 +628,33 @@ public void testAggUniqueDefaultValues() { intCol("IntCol", NULL_INT, 99999, 100000, 200000), longCol("LongCol", NULL_LONG, 44444444L, 55555555L, 66666666L), floatCol("FloatCol", NULL_FLOAT, 1.2345f, 2.3456f, 3.4567f), - doubleCol("DoubleCol", NULL_DOUBLE, 1.1E22d, 2.2E22d, 3.3E22d)); + doubleCol("DoubleCol", NULL_DOUBLE, 1.1E22d, 2.2E22d, 3.3E22d), + c("BigIntCol", null, + BigInteger.valueOf(Long.MAX_VALUE).add(BigInteger.valueOf(2)), + BigInteger.valueOf(Long.MIN_VALUE).subtract(BigInteger.valueOf(1)), + BigInteger.valueOf(Long.MIN_VALUE).subtract(BigInteger.valueOf(2))), + c("BigDecCol", null, + BigDecimal.valueOf(MAX_FINITE_DOUBLE).add(BigDecimal.valueOf(2)), + BigDecimal.valueOf(MIN_FINITE_DOUBLE).subtract(BigDecimal.valueOf(1)), + BigDecimal.valueOf(MIN_FINITE_DOUBLE).subtract(BigDecimal.valueOf(2)))); // First try mixing column types and values expectException(IllegalArgumentException.class, "Attempted to use no key/non unique values of incorrect types for aggregated columns!", () -> dataTable.aggBy(AggUnique(false, Sentinel(2), "StringCol", "BoolCol", "DatTime", "CharCol", - "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol"), "USym").sort("USym")); + "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", "DoubleCol", "BigIntCol", + "BigDecCol"), "USym").sort("USym")); dataTable.aggBy(AggUnique(false, Sentinel(-2), "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", - "DoubleCol"), "USym").sort("USym"); + "DoubleCol", "BigIntCol", "BigDecCol"), "USym").sort("USym"); + + dataTable.aggBy(AggUnique(false, Sentinel(BigInteger.valueOf(-2)), + "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", + "DoubleCol", "BigIntCol", "BigDecCol"), "USym").sort("USym"); + + dataTable.aggBy(AggUnique(false, Sentinel(BigDecimal.valueOf(-2)), + "ByteCol", "ShortCol", "IntCol", "LongCol", "FloatCol", + "DoubleCol", "BigIntCol", "BigDecCol"), "USym").sort("USym"); // Byte out of range testUniqueOutOfRangeParams(Byte.class, dataTable, ((short) Byte.MIN_VALUE - 1), Byte.MIN_VALUE, @@ -650,6 +668,9 @@ public void testAggUniqueDefaultValues() { Long.MIN_VALUE, BigInteger.valueOf(Long.MAX_VALUE).add(BigInteger.ONE), Long.MAX_VALUE, "LongCol", "FloatCol", "DoubleCol"); + testUniqueOutOfRangeParams(Long.class, dataTable, -2.2, + Long.MIN_VALUE, BigInteger.valueOf(Long.MAX_VALUE).add(BigInteger.ONE), Long.MAX_VALUE, "LongCol", + "FloatCol", "DoubleCol"); } private void testUniqueOutOfRangeParams(Class type, Table dataTable, Number invalidLow, Number validLow, From a02e95ed260fd056ffd8d7a87ee4255ce6512ee0 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Tue, 1 Feb 2022 22:43:52 -0500 Subject: [PATCH 39/44] WIP on JavaDoc improvements --- .../io/deephaven/api/agg/Aggregation.java | 129 +++++++++++++++++- .../deephaven/api/agg/ColumnAggregation.java | 3 +- .../deephaven/api/agg/ColumnAggregations.java | 3 +- .../main/java/io/deephaven/api/agg/Count.java | 6 + .../io/deephaven/api/agg/FirstRowKey.java | 8 ++ .../java/io/deephaven/api/agg/LastRowKey.java | 8 ++ 6 files changed, 154 insertions(+), 3 deletions(-) diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 95f72766403..98cbb1c1a7c 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -2,7 +2,6 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.agg.spec.AggSpec; -import io.deephaven.api.agg.spec.AggSpecFreeze; import io.deephaven.api.agg.util.PercentileOutput; import io.deephaven.api.agg.util.Sentinel; @@ -24,10 +23,25 @@ */ public interface Aggregation extends Serializable { + /** + * Combine an {@link AggSpec} and a {@link Pair column name pair} into a {@link ColumnAggregation}. + * + * @param spec The aggregation specifier to apply to the column pair + * @param pair The input/output column name pair + * @return The aggregation + */ static ColumnAggregation of(AggSpec spec, String pair) { return ColumnAggregation.of(spec, Pair.parse(pair)); } + /** + * Combine an {@link AggSpec} and one or more {@link Pair column name pairs} into a {@link ColumnAggregation} or + * {@link ColumnAggregations}. + * + * @param spec The aggregation specifier to apply to the column pair + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation of(AggSpec spec, String... pairs) { if (pairs.length == 1) { return of(spec, pairs[0]); @@ -39,6 +53,12 @@ static Aggregation of(AggSpec spec, String... pairs) { return builder.build(); } + /** + * Pass through a single {@link Aggregation}, or combine many into an {@link Aggregations}. + * + * @param aggregations The aggregations to combine + * @return The combined aggregation + */ static Aggregation of(Aggregation... aggregations) { if (aggregations.length == 1) { return aggregations[0]; @@ -46,6 +66,15 @@ static Aggregation of(Aggregation... aggregations) { return Aggregations.builder().addAggregations(aggregations).build(); } + /** + * Create a single or compound {@link Aggregation} from a single input column and one or more per-aggregation input + * values. + * + * @param columnAggFactory A factory for combining an input column and input value into an aggregation + * @param inputColumn The input column for each component of the resulting aggregation + * @param inputs The input values to combine with the input column via the factory + * @return The combined aggregation + */ @SafeVarargs static Aggregation of(BiFunction columnAggFactory, String inputColumn, INPUT_TYPE... inputs) { @@ -60,24 +89,66 @@ static Aggregation of(BiFunction aggFactory = (ic, po) -> ColumnAggregation .of(AggSpec.approximatePercentile(po.percentile()), Pair.of(ic, po.output())); return of(aggFactory, inputColumn, percentileOutputs); } + /** + * Create an {@link io.deephaven.api.agg.spec.AggSpecApproximatePercentile approximate percentile} aggregation for + * the supplied input column name, T-Digest compression factor, and percentile/output column name pairs. + * + * @param inputColumn The input column name + * @param compression T-Digest compression factor; must be > 1, should probably be < 1000 + * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @return THe aggregation + */ static Aggregation AggApproxPct(String inputColumn, double compression, PercentileOutput... percentileOutputs) { final BiFunction aggFactory = (ic, po) -> ColumnAggregation.of(AggSpec.approximatePercentile(po.percentile(), compression), @@ -233,27 +304,83 @@ static PercentileOutput PctOut(double percentile, String outputColumn) { return PercentileOutput.of(percentile, outputColumn); } + /** + * Make a {@link Sentinel sentinel} wrapping {@code value}. This serves to avoid ambiguity in the var-args overloads + * of some Aggregation factory methods. + * + * @param value The value to wrap + * @return The sentinel + * @see #AggUnique(boolean, Sentinel, String...) + */ static Sentinel Sentinel(Object value) { return Sentinel.of(value); } + /** + * Make a {@link Sentinel sentinel} wrapping {@code null}. This serves to avoid ambiguity in the var-args overloads + * of some Aggregation factory methods. + * + * @return The sentinel + * @see #AggUnique(boolean, Sentinel, String...) + */ static Sentinel Sentinel() { return Sentinel.of(); } + /** + * Glue method to deliver this Aggregation to a visitor. + * + * @param visitor The visitor + * @return The visitor + */ V walk(V visitor); + /** + * Visitor interface. Combines with {@link #walk(Visitor)} in order to allow for type-safe Aggregation evaluation + * without switch statements or if-else blocks. + */ interface Visitor { + + /** + * Visit a {@link Aggregations compound aggregation}. + * + * @param aggregations The compound aggregation to visit + */ void visit(Aggregations aggregations); + /** + * Visit a {@link ColumnAggregation column aggregation}. + * + * @param columnAgg The column aggregation to visit + */ void visit(ColumnAggregation columnAgg); + /** + * Visit a {@link ColumnAggregations compound column aggregation}. + * + * @param columnAggs The compound column aggregation to visit + */ void visit(ColumnAggregations columnAggs); + /** + * Visit a {@link Count count aggregation}. + * + * @param count The count aggregation + */ void visit(Count count); + /** + * Visit a {@link FirstRowKey first row key aggregation}. + * + * @param firstRowKey The first row key aggregation + */ void visit(FirstRowKey firstRowKey); + /** + * Visit a {@link LastRowKey last row key aggregation}. + * + * @param lastRowKey The last row key aggregation + */ void visit(LastRowKey lastRowKey); } } diff --git a/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java index aa0df65cfc2..20b04bb9b11 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregation.java @@ -7,7 +7,8 @@ /** * A ColumnAggregation is an {@link Aggregation} that is composed of a {@link #spec() spec} and a single input/output - * column {@link #pair() pair}. + * column {@link #pair() pair}. The spec defines the aggregation operation to apply to the input column in order to + * produce the paired output column. */ @Immutable @SimpleStyle diff --git a/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java index 43432f7db90..85e75bd98dd 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java +++ b/table-api/src/main/java/io/deephaven/api/agg/ColumnAggregations.java @@ -9,7 +9,8 @@ /** * ColumnAggregations is an {@link Aggregation} that is composed of a {@link #spec() spec} and multiple input/output - * column {@link #pairs() pairs}. + * column {@link #pairs() pairs}. The spec defines the aggregation operation to apply to each input column in order to + * produce the paired output column. */ @Immutable @BuildableStyle diff --git a/table-api/src/main/java/io/deephaven/api/agg/Count.java b/table-api/src/main/java/io/deephaven/api/agg/Count.java index 35960cde3f4..12d3c7c53f0 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Count.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Count.java @@ -5,6 +5,12 @@ import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Parameter; +/** + * An {@link Aggregation aggregation} that provides a single output column with the number of rows in each aggregation + * group. + * + * @see io.deephaven.api.TableOperations#countBy + */ @Immutable @SimpleStyle public abstract class Count implements Aggregation { diff --git a/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java b/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java index 68b6d707bb7..7d0c43e140d 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java +++ b/table-api/src/main/java/io/deephaven/api/agg/FirstRowKey.java @@ -5,6 +5,14 @@ import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Parameter; +/** + * An {@link Aggregation aggregation} that provides a single output column with the first row key from the input table + * for each aggregation group in the result. + *

+ * The primary use case for this aggregation is to allow for a subsequent {@link io.deephaven.api.TableOperations#sort + * sort} on the output column to order aggregated data by current first occurrence in the input table rather than + * encounter order. + */ @Immutable @SimpleStyle public abstract class FirstRowKey implements Aggregation { diff --git a/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java b/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java index 272342dc0a7..3edc159d831 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java +++ b/table-api/src/main/java/io/deephaven/api/agg/LastRowKey.java @@ -5,6 +5,14 @@ import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Parameter; +/** + * An {@link Aggregation aggregation} that provides a single output column with the last row key from the input table + * for each aggregation group in the result. + *

+ * The primary use case for this aggregation is to allow for a subsequent {@link io.deephaven.api.TableOperations#sort + * sort} on the output column to order aggregated data by current last occurrence in the input table rather than + * encounter order. + */ @Immutable @SimpleStyle public abstract class LastRowKey implements Aggregation { From 5363fdd56a2f8c2ea4eeb65da8a038c8b2e0cfee Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 2 Feb 2022 09:43:18 -0500 Subject: [PATCH 40/44] Spotless --- .../engine/table/impl/by/AggregationProcessor.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 7a19cccded8..ae58356a1f0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -1599,6 +1599,7 @@ private static Object maybeConvertType(@NotNull Class expected, final Object private interface NumericConverter { BigInteger toBigInteger(@Nullable final Number value); + BigDecimal toBigDecimal(@Nullable final Number value); private static NumericConverter lookup(@NotNull final Class numberClass) { @@ -1624,8 +1625,8 @@ private enum IntegralType implements NumericConverter { private final BigInteger upperBound; IntegralType(@NotNull final Function toBigInteger, - @Nullable final Number lowerBound, - @Nullable final Number upperBound) { + @Nullable final Number lowerBound, + @Nullable final Number upperBound) { this.toBigInteger = toBigInteger; this.lowerBound = toBigInteger(lowerBound); this.upperBound = toBigInteger(upperBound); @@ -1670,8 +1671,8 @@ private enum FloatingPointType implements NumericConverter { private final BigDecimal upperBound; FloatingPointType(@NotNull final Function toBigDecimal, - @Nullable final Number lowerBound, - @Nullable final Number upperBound) { + @Nullable final Number lowerBound, + @Nullable final Number upperBound) { this.toBigDecimal = toBigDecimal; this.lowerBound = toBigDecimal(lowerBound); this.upperBound = toBigDecimal(upperBound); @@ -1705,7 +1706,7 @@ private static FloatingPointType lookup(@NotNull final Class numberClass) { } private static boolean isNumericallyCompatible(@NotNull final Number value, - @NotNull final Class expected) { + @NotNull final Class expected) { final NumericConverter valueConverter = NumericConverter.lookup(value.getClass()); if (valueConverter == null) { // value is not a recognized type From 7b0269b9384ed88764fdbf3e015529d8e413cac6 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Wed, 2 Feb 2022 18:20:31 -0500 Subject: [PATCH 41/44] Further WIP on JavaDoc --- .../io/deephaven/api/agg/Aggregation.java | 192 +++++++++++++++++- .../io/deephaven/api/agg/spec/AggSpec.java | 134 +++++++++++- .../deephaven/api/agg/spec/AggSpecAbsSum.java | 3 + 3 files changed, 322 insertions(+), 7 deletions(-) diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 98cbb1c1a7c..bf9067be4e9 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -94,7 +94,7 @@ static Aggregation of(BiFunction aggFactory = (ic, po) -> ColumnAggregation @@ -147,7 +147,7 @@ static Aggregation AggApproxPct(String inputColumn, PercentileOutput... percenti * @param inputColumn The input column name * @param compression T-Digest compression factor; must be > 1, should probably be < 1000 * @param percentileOutputs The percentile/output column name pairs for the component aggregations - * @return THe aggregation + * @return The aggregation */ static Aggregation AggApproxPct(String inputColumn, double compression, PercentileOutput... percentileOutputs) { final BiFunction aggFactory = @@ -156,88 +156,240 @@ static Aggregation AggApproxPct(String inputColumn, double compression, Percenti return of(aggFactory, inputColumn, percentileOutputs); } + /** + * Create an {@link io.deephaven.api.agg.spec.AggSpecAvg average} (arithmetic mean) aggregation for the + * supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggAvg(String... pairs) { return of(AggSpec.avg(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.Count count} aggregation with the supplied output column name. + * + * @param resultColumn The output column name + * @return The aggregation + */ static Aggregation AggCount(String resultColumn) { return Count.of(resultColumn); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecCountDistinct count distinct} aggregation for the supplied + * column name pairs. This will not count {@code null} values from the input column(s). + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggCountDistinct(String... pairs) { return of(AggSpec.countDistinct(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecCountDistinct count distinct} aggregation for the supplied + * column name pairs. This will count {@code null} values from the input column(s) if {@code countNulls} is + * {@code true}. + * + * @param countNulls Whether {@code null} values should be counted + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggCountDistinct(boolean countNulls, String... pairs) { return of(AggSpec.countDistinct(countNulls), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecDistinct distinct} aggregation for the supplied column name + * pairs. This will not include {@code null} values in the output column(s). + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggDistinct(String... pairs) { return of(AggSpec.distinct(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecDistinct distinct} aggregation for the supplied column name + * pairs. This will include {@code null} values it the output column(s) if {@code includeNulls} is {@code true}. + * + * @param includeNulls Whether {@code null} values should be included + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggDistinct(boolean includeNulls, String... pairs) { return of(AggSpec.distinct(includeNulls), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecFirst first} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggFirst(String... pairs) { return of(AggSpec.first(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.FirstRowKey first row key} aggregation with the supplied result column. + * + * @param resultColumn The output column name + * @return The aggregation + */ static Aggregation AggFirstRowKey(String resultColumn) { return FirstRowKey.of(resultColumn); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecFormula formula} aggregation with the supplied {@code formula}, + * {@code formulaParam}, and column name pairs. + * + * @param formula The formula to use for all input columns to produce all output columns + * @param formulaParam The token to replace with the input column name in {@code formula} + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggFormula(String formula, String formulaParam, String... pairs) { return of(AggSpec.formula(formula, formulaParam), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecFreeze freeze} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggFreeze(String... pairs) { return of(AggSpec.freeze(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecGroup group} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggGroup(String... pairs) { return of(AggSpec.group(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecLast last} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggLast(String... pairs) { return of(AggSpec.last(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.LastRowKey last row key} aggregation with the supplied result column. + * + * @param resultColumn The output column name + * @return The aggregation + */ static Aggregation AggLastRowKey(String resultColumn) { return LastRowKey.of(resultColumn); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecMax max} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggMax(String... pairs) { return of(AggSpec.max(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecMedian median} aggregation for the supplied column name pairs. + * For numeric types, if there are an even number of values the result will be an average of the two middle values. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggMed(String... pairs) { return of(AggSpec.median(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecMedian median} aggregation for the supplied column name pairs. + * For numeric types, if there are an even number of values the result will be an average of the two middle values + * if {@code average} is {@code true}, else the result will be the lower of the two middle values. + * + * @param average Whether to average the middle two values for even-sized result sets of numeric types + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggMed(boolean average, String... pairs) { return of(AggSpec.median(average), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecMin min} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggMin(String... pairs) { return of(AggSpec.min(), pairs); } + /** + * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied percentile + * and column name pairs. + * + * @param percentile The percentile to use for all component aggregations + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggPct(double percentile, String... pairs) { return of(AggSpec.percentile(percentile), pairs); } + /** + * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied percentile + * and column name pairs. If the percentile equally divides the value space, the result will be the average of the + * values immediately below an above if {@code average} is {@code true}. + * + * @param percentile The percentile to use for all component aggregations + * @param average + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggPct(double percentile, boolean average, String... pairs) { return of(AggSpec.percentile(percentile, average), pairs); } + /** + * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied input + * column name and percentile/output column name pairs. + * + * @param inputColumn The input column name + * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @return The aggregation + */ static Aggregation AggPct(String inputColumn, PercentileOutput... percentileOutputs) { final BiFunction aggFactory = (ic, po) -> ColumnAggregation.of(AggSpec.percentile(po.percentile()), Pair.of(ic, po.output())); return of(aggFactory, inputColumn, percentileOutputs); } + /** + * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied input + * column name and percentile/output column name pairs. If the percentile equally divides the value space, the + * result will be the average of the values immediately below an above if {@code average} is {@code true}. + * + * @param inputColumn The input column name + * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @return The aggregation + */ static Aggregation AggPct(String inputColumn, boolean average, PercentileOutput... percentileOutputs) { final BiFunction aggFactory = (ic, po) -> ColumnAggregation .of(AggSpec.percentile(po.percentile(), average), Pair.of(ic, po.output())); @@ -276,14 +428,42 @@ static Aggregation AggTDigest(double compression, String... pairs) { return of(AggSpec.tDigest(compression), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecUnique unique} aggregation for the supplied column name pairs. + * This will not consider {@code null} values when determining if a group has a single unique value. Non-unique + * groups will have {@code null} values in the output column. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggUnique(String... pairs) { return of(AggSpec.unique(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecUnique unique} aggregation for the supplied column name pairs. + * This will consider {@code null} values when determining if a group has a single unique value if + * {@code includeNulls} is {@code true}. Non-unique groups will have {@code null} values in the output column. + * + * @param includeNulls Whether to consider {@code null} values towards uniqueness + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggUnique(boolean includeNulls, String... pairs) { return AggUnique(includeNulls, Sentinel.of(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecUnique unique} aggregation for the supplied column name pairs. + * This will consider {@code null} values when determining if a group has a single unique value if + * {@code includeNulls} is {@code true}. Non-unique groups will have the value wrapped by {@code nonUniqueSentinel} + * in the output column. + * + * @param includeNulls Whether to consider {@code null} values towards uniqueness + * @param nonUniqueSentinel The value to output for non-unique groups + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggUnique(boolean includeNulls, Sentinel nonUniqueSentinel, String... pairs) { return of(AggSpec.unique(includeNulls, nonUniqueSentinel.value()), pairs); } @@ -328,7 +508,7 @@ static Sentinel Sentinel() { } /** - * Glue method to deliver this Aggregation to a visitor. + * Glue method to deliver this Aggregation to a {@link AggSpec.Visitor}. * * @param visitor The visitor * @return The visitor diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 22643d255c0..541927938da 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -10,22 +10,35 @@ import java.util.Collection; /** - * An aggregation specification represents the configurable options for column aggregations. + * An aggregation specification represents the configurable options for {@link ColumnAggregation singular} and + * {@link io.deephaven.api.agg.ColumnAggregations compound} column aggregations. */ public interface AggSpec { + /** + * @return {@link AggSpecAbsSum#of()} + */ static AggSpecAbsSum absSum() { return AggSpecAbsSum.of(); } + /** + * @return {@link AggSpecApproximatePercentile#of(double)} for {@code percentile} + */ static AggSpecApproximatePercentile approximatePercentile(double percentile) { return AggSpecApproximatePercentile.of(percentile); } + /** + * @return {@link AggSpecApproximatePercentile#of(double, double)} for {@code percentile} and {@code compression} + */ static AggSpecApproximatePercentile approximatePercentile(double percentile, double compression) { return AggSpecApproximatePercentile.of(percentile, compression); } + /** + * @return {@link AggSpecAvg#of()} + */ static AggSpecAvg avg() { return AggSpecAvg.of(); } @@ -34,70 +47,121 @@ static AggSpecCountDistinct countDistinct() { return AggSpecCountDistinct.of(); } + /** + * @return {@link AggSpecCountDistinct#of(boolean)} for {@code countNulls} + */ static AggSpecCountDistinct countDistinct(boolean countNulls) { return AggSpecCountDistinct.of(countNulls); } + /** + * @return {@link AggSpecDistinct#of()} + */ static AggSpecDistinct distinct() { return AggSpecDistinct.of(); } + /** + * @return {@link AggSpecDistinct#of(boolean)} for {@code includeNulls} + */ static AggSpecDistinct distinct(boolean includeNulls) { return AggSpecDistinct.of(includeNulls); } + /** + * @return {@link AggSpecFirst#of()} + */ static AggSpecFirst first() { return AggSpecFirst.of(); } + /** + * @return {@link AggSpecFormula#of(String)} for {@code formula} + */ static AggSpecFormula formula(String formula) { return AggSpecFormula.of(formula); } + /** + * @return {@link AggSpecFormula#of(String, String)} for {@code formula} and {@code formulaParam} + */ static AggSpecFormula formula(String formula, String formulaParam) { return AggSpecFormula.of(formula, formulaParam); } + /** + * @return {@link AggSpecFreeze#of()} + */ static AggSpecFreeze freeze() { return AggSpecFreeze.of(); } + /** + * @return {@link AggSpecGroup#of()} + */ static AggSpecGroup group() { return AggSpecGroup.of(); } + /** + * @return {@link AggSpecLast#of()} + */ static AggSpecLast last() { return AggSpecLast.of(); } + /** + * @return {@link AggSpecMax#of()} + */ static AggSpecMax max() { return AggSpecMax.of(); } + /** + * @return {@link AggSpecMedian#of()} + */ static AggSpecMedian median() { return AggSpecMedian.of(); } + /** + * @return {@link AggSpecMedian#of(boolean)} for {@code averageMedian} + */ static AggSpecMedian median(boolean averageMedian) { return AggSpecMedian.of(averageMedian); } + /** + * @return {@link AggSpecMin#of()} + */ static AggSpecMin min() { return AggSpecMin.of(); } + /** + * @return {@link AggSpecPercentile#of(double)} for {@code percentile} + */ static AggSpecPercentile percentile(double percentile) { return AggSpecPercentile.of(percentile); } + /** + * @return {@link AggSpecPercentile#of(double, boolean)} for {@code percentile} and {@code averageMedian} + */ static AggSpecPercentile percentile(double percentile, boolean averageMedian) { return AggSpecPercentile.of(percentile, averageMedian); } + /** + * @return {@link AggSpecSortedFirst} for the supplied sort {@code columns} + */ static AggSpecSortedFirst sortedFirst(String... columns) { return sortedFirst(Arrays.asList(columns)); } + /** + * @return {@link AggSpecSortedFirst} for the supplied sort {@code columns} + */ static AggSpecSortedFirst sortedFirst(Collection columns) { final AggSpecSortedFirst.Builder builder = AggSpecSortedFirst.builder(); for (String column : columns) { @@ -106,10 +170,16 @@ static AggSpecSortedFirst sortedFirst(Collection columns) { return builder.build(); } + /** + * @return {@link AggSpecSortedLast} for the supplied sort {@code columns} + */ static AggSpecSortedLast sortedLast(String... columns) { return sortedLast(Arrays.asList(columns)); } + /** + * @return {@link AggSpecSortedLast} for the supplied sort {@code columns} + */ static AggSpecSortedLast sortedLast(Collection columns) { final AggSpecSortedLast.Builder builder = AggSpecSortedLast.builder(); for (String column : columns) { @@ -118,52 +188,114 @@ static AggSpecSortedLast sortedLast(Collection columns) { return builder.build(); } + /** + * @return {@link AggSpecStd#of()} + */ static AggSpecStd std() { return AggSpecStd.of(); } + /** + * @return {@link AggSpecSum#of()} + */ static AggSpecSum sum() { return AggSpecSum.of(); } + /** + * @return {@link AggSpecTDigest#of()} + */ static AggSpecTDigest tDigest() { return AggSpecTDigest.of(); } + /** + * @return {@link AggSpecTDigest#of(double)} for {@code compression} + */ static AggSpecTDigest tDigest(double compression) { return AggSpecTDigest.of(compression); } + /** + * @return {@link AggSpecUnique#of()} + */ static AggSpecUnique unique() { return AggSpecUnique.of(); } + /** + * @return {@link AggSpecUnique#of(boolean, Object)} for {@code includeNulls} and {@code nonUniqueSentinel} + */ static AggSpecUnique unique(boolean includeNulls, Object nonUniqueSentinel) { return AggSpecUnique.of(includeNulls, nonUniqueSentinel); } + /** + * @return {@link AggSpecVar#of()} + */ static AggSpecVar var() { return AggSpecVar.of(); } + /** + * @return {@link AggSpecWAvg#of(ColumnName)} for the supplied {@code weightColumn} + */ static AggSpecWAvg wavg(String weightColumn) { return AggSpecWAvg.of(ColumnName.of(weightColumn)); } + /** + * @return {@link AggSpecWSum#of(ColumnName)} for the supplied {@code weightColumn} + */ static AggSpecWSum wsum(String weightColumn) { return AggSpecWSum.of(ColumnName.of(weightColumn)); } + /** + * Build a {@link ColumnAggregation} for this AggSpec. + * + * @param pair The input/output column name pair + * @return The aggregation + */ ColumnAggregation aggregation(Pair pair); + /** + * Build a {@link ColumnAggregation singular} or {@link io.deephaven.api.agg.ColumnAggregations compound} + * aggregation for this AggSpec. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ Aggregation aggregation(Pair... pairs); + /** + * Build a {@link ColumnAggregation singular} or {@link io.deephaven.api.agg.ColumnAggregations compound} + * aggregation for this AggSpec. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ Aggregation aggregation(Collection pairs); + /** + * Get a simple description for this AggSpec. + * + * @return The description + */ String description(); + /** + * Glue method to deliver this AggSpec to a {@link Visitor}. + * + * @param visitor The visitor + * @return The visitor + */ V walk(V visitor); + /* + * Visitor interface. Combines with {@link #walk(Visitor) in order to allow for type-safe AggSpec evaluation + * without switch statements or if-else blocks. + */ interface Visitor { void visit(AggSpecAbsSum absSum); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java index 4dbc0f4a7ab..70f02377676 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java @@ -3,6 +3,9 @@ import io.deephaven.annotations.SimpleStyle; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that supplies the absolute sum for each group. Only works with numeric input types. + */ @Immutable @SimpleStyle public abstract class AggSpecAbsSum extends AggSpecEmptyBase { From de632c7f3a854cb897b0b3c8412ddddbd70425a5 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 3 Feb 2022 10:15:07 -0500 Subject: [PATCH 42/44] Rename "averageMedian" to "averageEvenlyDivided" --- .../BytePercentileTypeMedianHelper.java | 4 ++-- .../CharPercentileTypeMedianHelper.java | 4 ++-- .../ssmpercentile/IntPercentileTypeMedianHelper.java | 4 ++-- .../LongPercentileTypeMedianHelper.java | 4 ++-- .../ShortPercentileTypeMedianHelper.java | 4 ++-- .../ssmpercentile/SsmChunkedPercentileOperator.java | 12 ++++++------ 6 files changed, 16 insertions(+), 16 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/BytePercentileTypeMedianHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/BytePercentileTypeMedianHelper.java index 49e8c23b429..176b374c07c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/BytePercentileTypeMedianHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/BytePercentileTypeMedianHelper.java @@ -42,9 +42,9 @@ public boolean setResult(SegmentedSortedMultiSet ssmLo, SegmentedSortedMultiSet } if (ssmLo.totalSize() == ssmHi.totalSize()) { - // region averageMedian + // region averageEvenlyDivided return setResult(destination, (((ByteSegmentedSortedMultiset)ssmLo).getMaxByte() + ((ByteSegmentedSortedMultiset)ssmHi).getMinByte()) / 2.0); - // endregion averageMedian + // endregion averageEvenlyDivided } else { return setResult(destination, ((ByteSegmentedSortedMultiset)ssmLo).getMaxByte()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/CharPercentileTypeMedianHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/CharPercentileTypeMedianHelper.java index 762b3d631c6..d5e990771ca 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/CharPercentileTypeMedianHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/CharPercentileTypeMedianHelper.java @@ -37,9 +37,9 @@ public boolean setResult(SegmentedSortedMultiSet ssmLo, SegmentedSortedMultiSet } if (ssmLo.totalSize() == ssmHi.totalSize()) { - // region averageMedian + // region averageEvenlyDivided return setResult(destination, (((CharSegmentedSortedMultiset)ssmLo).getMaxChar() + ((CharSegmentedSortedMultiset)ssmHi).getMinChar()) / 2.0); - // endregion averageMedian + // endregion averageEvenlyDivided } else { return setResult(destination, ((CharSegmentedSortedMultiset)ssmLo).getMaxChar()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/IntPercentileTypeMedianHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/IntPercentileTypeMedianHelper.java index 1423122e176..b3cd956e77c 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/IntPercentileTypeMedianHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/IntPercentileTypeMedianHelper.java @@ -42,9 +42,9 @@ public boolean setResult(SegmentedSortedMultiSet ssmLo, SegmentedSortedMultiSet } if (ssmLo.totalSize() == ssmHi.totalSize()) { - // region averageMedian + // region averageEvenlyDivided return setResult(destination, (((IntSegmentedSortedMultiset)ssmLo).getMaxInt() + ((IntSegmentedSortedMultiset)ssmHi).getMinInt()) / 2.0); - // endregion averageMedian + // endregion averageEvenlyDivided } else { return setResult(destination, ((IntSegmentedSortedMultiset)ssmLo).getMaxInt()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/LongPercentileTypeMedianHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/LongPercentileTypeMedianHelper.java index ca6a388f3b3..ee7d96566de 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/LongPercentileTypeMedianHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/LongPercentileTypeMedianHelper.java @@ -42,9 +42,9 @@ public boolean setResult(SegmentedSortedMultiSet ssmLo, SegmentedSortedMultiSet } if (ssmLo.totalSize() == ssmHi.totalSize()) { - // region averageMedian + // region averageEvenlyDivided return setResult(destination, (((LongSegmentedSortedMultiset)ssmLo).getMaxLong() + ((LongSegmentedSortedMultiset)ssmHi).getMinLong()) / 2.0); - // endregion averageMedian + // endregion averageEvenlyDivided } else { return setResult(destination, ((LongSegmentedSortedMultiset)ssmLo).getMaxLong()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/ShortPercentileTypeMedianHelper.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/ShortPercentileTypeMedianHelper.java index d445988670a..2c6c41e1855 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/ShortPercentileTypeMedianHelper.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/ShortPercentileTypeMedianHelper.java @@ -42,9 +42,9 @@ public boolean setResult(SegmentedSortedMultiSet ssmLo, SegmentedSortedMultiSet } if (ssmLo.totalSize() == ssmHi.totalSize()) { - // region averageMedian + // region averageEvenlyDivided return setResult(destination, (((ShortSegmentedSortedMultiset)ssmLo).getMaxShort() + ((ShortSegmentedSortedMultiset)ssmHi).getMinShort()) / 2.0); - // endregion averageMedian + // endregion averageEvenlyDivided } else { return setResult(destination, ((ShortSegmentedSortedMultiset)ssmLo).getMaxShort()); } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/SsmChunkedPercentileOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/SsmChunkedPercentileOperator.java index 1a5ca156f1a..6ee838277e9 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/SsmChunkedPercentileOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/ssmpercentile/SsmChunkedPercentileOperator.java @@ -43,7 +43,7 @@ public class SsmChunkedPercentileOperator implements IterativeChunkedAggregation private final ChunkType chunkType; private final PercentileTypeHelper percentileTypeHelper; - public SsmChunkedPercentileOperator(Class type, double percentile, boolean averageMedian, String name) { + public SsmChunkedPercentileOperator(Class type, double percentile, boolean averageEvenlyDivided, String name) { this.name = name; this.ssms = new ObjectArraySource<>(SegmentedSortedMultiSet.class); final boolean isDateTime = type == DateTime.class; @@ -56,9 +56,9 @@ public SsmChunkedPercentileOperator(Class type, double percentile, boolean av internalResult = new LongArraySource(); // noinspection unchecked externalResult = new BoxedColumnSource.OfDateTime(internalResult); - averageMedian = false; + averageEvenlyDivided = false; } else { - if (averageMedian) { + if (averageEvenlyDivided) { switch (chunkType) { case Int: case Long: @@ -81,12 +81,12 @@ public SsmChunkedPercentileOperator(Class type, double percentile, boolean av compactAndCountKernel = CompactKernel.makeCompact(chunkType); ssmFactory = SegmentedSortedMultiSet.makeFactory(chunkType, NODE_SIZE, type); removeContextFactory = SegmentedSortedMultiSet.makeRemoveContextFactory(NODE_SIZE); - percentileTypeHelper = makeTypeHelper(chunkType, type, percentile, averageMedian, internalResult); + percentileTypeHelper = makeTypeHelper(chunkType, type, percentile, averageEvenlyDivided, internalResult); } private static PercentileTypeHelper makeTypeHelper(ChunkType chunkType, Class type, double percentile, - boolean averageMedian, ArrayBackedColumnSource resultColumn) { - if (averageMedian) { + boolean averageEvenlyDivided, ArrayBackedColumnSource resultColumn) { + if (averageEvenlyDivided) { switch (chunkType) { // for things that are not int, long, double, or float we do not actually average the median; // we just do the standard 50-%tile thing. It might be worth defining this to be friendlier. From 8c920116e8b71b31e361fdeb5ec48d5b662fdb5a Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 3 Feb 2022 12:37:55 -0500 Subject: [PATCH 43/44] Finish JavaDoc cleanup --- .../table/impl/by/AggregationProcessor.java | 6 +- .../client/impl/BatchTableRequestBuilder.java | 4 +- .../io/deephaven/api/agg/Aggregation.java | 227 ++++++++++++++---- .../io/deephaven/api/agg/spec/AggSpec.java | 18 +- .../deephaven/api/agg/spec/AggSpecAbsSum.java | 13 +- .../spec/AggSpecApproximatePercentile.java | 10 +- .../io/deephaven/api/agg/spec/AggSpecAvg.java | 5 + .../api/agg/spec/AggSpecCountDistinct.java | 8 + .../api/agg/spec/AggSpecDistinct.java | 9 + .../deephaven/api/agg/spec/AggSpecFirst.java | 6 + .../api/agg/spec/AggSpecFormula.java | 28 ++- .../deephaven/api/agg/spec/AggSpecFreeze.java | 12 +- .../deephaven/api/agg/spec/AggSpecGroup.java | 7 + .../deephaven/api/agg/spec/AggSpecLast.java | 6 + .../io/deephaven/api/agg/spec/AggSpecMax.java | 7 + .../deephaven/api/agg/spec/AggSpecMedian.java | 18 +- .../io/deephaven/api/agg/spec/AggSpecMin.java | 7 + .../api/agg/spec/AggSpecPercentile.java | 26 +- .../api/agg/spec/AggSpecSortedFirst.java | 11 +- .../api/agg/spec/AggSpecSortedLast.java | 9 +- .../io/deephaven/api/agg/spec/AggSpecStd.java | 7 + .../io/deephaven/api/agg/spec/AggSpecSum.java | 14 ++ .../api/agg/spec/AggSpecTDigest.java | 4 +- .../deephaven/api/agg/spec/AggSpecUnique.java | 15 ++ .../io/deephaven/api/agg/spec/AggSpecVar.java | 7 + .../deephaven/api/agg/spec/AggSpecWAvg.java | 12 + .../deephaven/api/agg/spec/AggSpecWSum.java | 12 + 27 files changed, 411 insertions(+), 97 deletions(-) diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index ae58356a1f0..4d36f504f76 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -689,7 +689,7 @@ public void visit(@NotNull final AggSpecFormula formula) { final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, resultPairs.stream().map(pair -> MatchPair.of((Pair) pair.input())).toArray(MatchPair[]::new)); final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator(groupByChunkedOperator, - true, formula.formula(), formula.formulaParam(), MatchPair.fromPairs(resultPairs)); + true, formula.formula(), formula.paramToken(), MatchPair.fromPairs(resultPairs)); addNoInputOperator(formulaChunkedOperator); } @@ -716,7 +716,7 @@ public void visit(@NotNull final AggSpecMax max) { @Override public void visit(@NotNull final AggSpecMedian median) { - addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, 0.50d, median.averageMedian(), n)); + addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, 0.50d, median.averageEvenlyDivided(), n)); } @Override @@ -726,7 +726,7 @@ public void visit(@NotNull final AggSpecMin min) { @Override public void visit(@NotNull final AggSpecPercentile pct) { - addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, pct.percentile(), pct.averageMedian(), n)); + addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, pct.percentile(), pct.averageEvenlyDivided(), n)); } @Override diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java index ca4bb92dfb0..fa2339c91f5 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/BatchTableRequestBuilder.java @@ -742,7 +742,7 @@ public void visit(AggSpecMax max) { @Override public void visit(AggSpecMedian median) { - if (!median.averageMedian()) { + if (!median.averageEvenlyDivided()) { throw new UnsupportedOperationException( "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } @@ -756,7 +756,7 @@ public void visit(AggSpecMin min) { @Override public void visit(AggSpecPercentile pct) { - if (pct.averageMedian()) { + if (pct.averageEvenlyDivided()) { throw new UnsupportedOperationException( "TODO(deephaven-core#991): TableService aggregation coverage, https://github.com/deephaven/deephaven-core/issues/991"); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index bf9067be4e9..b468a2d98c6 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -2,6 +2,16 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.api.agg.spec.AggSpecApproximatePercentile; +import io.deephaven.api.agg.spec.AggSpecCountDistinct; +import io.deephaven.api.agg.spec.AggSpecDistinct; +import io.deephaven.api.agg.spec.AggSpecFormula; +import io.deephaven.api.agg.spec.AggSpecMedian; +import io.deephaven.api.agg.spec.AggSpecPercentile; +import io.deephaven.api.agg.spec.AggSpecTDigest; +import io.deephaven.api.agg.spec.AggSpecUnique; +import io.deephaven.api.agg.spec.AggSpecWAvg; +import io.deephaven.api.agg.spec.AggSpecWSum; import io.deephaven.api.agg.util.PercentileOutput; import io.deephaven.api.agg.util.Sentinel; @@ -24,10 +34,10 @@ public interface Aggregation extends Serializable { /** - * Combine an {@link AggSpec} and a {@link Pair column name pair} into a {@link ColumnAggregation}. + * Combine an {@link AggSpec} and an input/output {@link Pair column name pair} into a {@link ColumnAggregation}. * - * @param spec The aggregation specifier to apply to the column pair - * @param pair The input/output column name pair + * @param spec The {@link ColumnAggregation#spec() aggregation specifier} to apply to the column name pair + * @param pair The {@link ColumnAggregation#pair() input/output column name pair} * @return The aggregation */ static ColumnAggregation of(AggSpec spec, String pair) { @@ -35,11 +45,12 @@ static ColumnAggregation of(AggSpec spec, String pair) { } /** - * Combine an {@link AggSpec} and one or more {@link Pair column name pairs} into a {@link ColumnAggregation} or - * {@link ColumnAggregations}. + * Combine an {@link AggSpec} and one or more input/output {@link Pair column name pairs} into a + * {@link ColumnAggregation} or {@link ColumnAggregations}. * - * @param spec The aggregation specifier to apply to the column pair - * @param pairs The input/output column name pairs + * @param spec The {@link ColumnAggregation#spec() aggregation specifier} to apply to the column name pair(s) + * @param pairs The input/output column name {@link ColumnAggregation#pair() pair} or + * {@link ColumnAggregations#pairs() pairs} * @return The aggregation */ static Aggregation of(AggSpec spec, String... pairs) { @@ -56,7 +67,7 @@ static Aggregation of(AggSpec spec, String... pairs) { /** * Pass through a single {@link Aggregation}, or combine many into an {@link Aggregations}. * - * @param aggregations The aggregations to combine + * @param aggregations The {@link Aggregations#aggregations() aggregations} to combine * @return The combined aggregation */ static Aggregation of(Aggregation... aggregations) { @@ -70,7 +81,8 @@ static Aggregation of(Aggregation... aggregations) { * Create a single or compound {@link Aggregation} from a single input column and one or more per-aggregation input * values. * - * @param columnAggFactory A factory for combining an input column and input value into an aggregation + * @param columnAggFactory A factory for combining an input column and input value into a {@link ColumnAggregation + * aggregation} * @param inputColumn The input column for each component of the resulting aggregation * @param inputs The input values to combine with the input column via the factory * @return The combined aggregation @@ -102,9 +114,11 @@ static Aggregation AggAbsSum(String... pairs) { /** * Create an {@link io.deephaven.api.agg.spec.AggSpecApproximatePercentile approximate percentile} aggregation for - * the supplied percentile and column name pairs with the default T-Digest compression factor. + * the supplied percentile and column name pairs with the default T-Digest + * {@link AggSpecApproximatePercentile#compression() compression} factor. * - * @param percentile The percentile to use for all component aggregations + * @param percentile The {@link AggSpecApproximatePercentile#percentile() percentile} to use for all component + * aggregations * @param pairs The input/output column name pairs * @return The aggregation */ @@ -116,8 +130,10 @@ static Aggregation AggApproxPct(double percentile, String... pairs) { * Create an {@link io.deephaven.api.agg.spec.AggSpecApproximatePercentile approximate percentile} aggregation for * the supplied percentile, T-Digest compression factor, and column name pairs. * - * @param percentile The percentile to use for all component aggregations - * @param compression T-Digest compression factor; must be > 1, should probably be < 1000 + * @param percentile The {@link AggSpecApproximatePercentile#percentile() percentile} to use for all component + * aggregations + * @param compression T-Digest {@link AggSpecTDigest#compression() compression} factor; must be > 1, should + * probably be < 1000 * @param pairs The input/output column name pairs * @return The aggregation */ @@ -127,11 +143,12 @@ static Aggregation AggApproxPct(double percentile, double compression, String... /** * Create an {@link io.deephaven.api.agg.spec.AggSpecApproximatePercentile approximate percentile} aggregation for - * the supplied input column name and percentile/output column name pairs with the default T-Digest compression - * factor. + * the supplied input column name and percentile/output column name pairs with the default T-Digest + * {@link AggSpecApproximatePercentile#compression() compression} factor. * * @param inputColumn The input column name - * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @param percentileOutputs The {@link PercentileOutput percentile/output column name pairs} for the component + * aggregations; see {@link #PctOut(double, String)}. * @return The aggregation */ static Aggregation AggApproxPct(String inputColumn, PercentileOutput... percentileOutputs) { @@ -145,8 +162,10 @@ static Aggregation AggApproxPct(String inputColumn, PercentileOutput... percenti * the supplied input column name, T-Digest compression factor, and percentile/output column name pairs. * * @param inputColumn The input column name - * @param compression T-Digest compression factor; must be > 1, should probably be < 1000 - * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @param compression T-Digest {@link AggSpecTDigest#compression() compression} factor; must be > 1, should + * probably be < 1000 + * @param percentileOutputs The {@link PercentileOutput percentile/output column name pairs} for the component + * aggregations; see {@link #PctOut(double, String)}. * @return The aggregation */ static Aggregation AggApproxPct(String inputColumn, double compression, PercentileOutput... percentileOutputs) { @@ -170,7 +189,7 @@ static Aggregation AggAvg(String... pairs) { /** * Create a {@link io.deephaven.api.agg.Count count} aggregation with the supplied output column name. * - * @param resultColumn The output column name + * @param resultColumn The {@link Count#column() output column} name * @return The aggregation */ static Aggregation AggCount(String resultColumn) { @@ -193,7 +212,7 @@ static Aggregation AggCountDistinct(String... pairs) { * column name pairs. This will count {@code null} values from the input column(s) if {@code countNulls} is * {@code true}. * - * @param countNulls Whether {@code null} values should be counted + * @param countNulls Whether {@code null} values should be counted; see {@link AggSpecCountDistinct#countNulls()}} * @param pairs The input/output column name pairs * @return The aggregation */ @@ -214,9 +233,9 @@ static Aggregation AggDistinct(String... pairs) { /** * Create a {@link io.deephaven.api.agg.spec.AggSpecDistinct distinct} aggregation for the supplied column name - * pairs. This will include {@code null} values it the output column(s) if {@code includeNulls} is {@code true}. + * pairs. This will include {@code null} values in the output column(s) if {@code includeNulls} is {@code true}. * - * @param includeNulls Whether {@code null} values should be included + * @param includeNulls Whether {@code null} values should be included; see {@link AggSpecDistinct#includeNulls()} * @param pairs The input/output column name pairs * @return The aggregation */ @@ -237,7 +256,7 @@ static Aggregation AggFirst(String... pairs) { /** * Create a {@link io.deephaven.api.agg.FirstRowKey first row key} aggregation with the supplied result column. * - * @param resultColumn The output column name + * @param resultColumn The {@link FirstRowKey#column() output column} name * @return The aggregation */ static Aggregation AggFirstRowKey(String resultColumn) { @@ -246,15 +265,17 @@ static Aggregation AggFirstRowKey(String resultColumn) { /** * Create a {@link io.deephaven.api.agg.spec.AggSpecFormula formula} aggregation with the supplied {@code formula}, - * {@code formulaParam}, and column name pairs. + * {@code paramToken}, and column name pairs. * - * @param formula The formula to use for all input columns to produce all output columns - * @param formulaParam The token to replace with the input column name in {@code formula} + * @param formula The {@link AggSpecFormula#formula() formula} to use for all input columns to produce all output + * columns + * @param paramToken The {@link AggSpecFormula#paramToken() parameter token} to replace with the input column + * name in {@code formula} * @param pairs The input/output column name pairs * @return The aggregation */ - static Aggregation AggFormula(String formula, String formulaParam, String... pairs) { - return of(AggSpec.formula(formula, formulaParam), pairs); + static Aggregation AggFormula(String formula, String paramToken, String... pairs) { + return of(AggSpec.formula(formula, paramToken), pairs); } /** @@ -290,7 +311,7 @@ static Aggregation AggLast(String... pairs) { /** * Create a {@link io.deephaven.api.agg.LastRowKey last row key} aggregation with the supplied result column. * - * @param resultColumn The output column name + * @param resultColumn The {@link LastRowKey#column() output column} name * @return The aggregation */ static Aggregation AggLastRowKey(String resultColumn) { @@ -323,7 +344,8 @@ static Aggregation AggMed(String... pairs) { * For numeric types, if there are an even number of values the result will be an average of the two middle values * if {@code average} is {@code true}, else the result will be the lower of the two middle values. * - * @param average Whether to average the middle two values for even-sized result sets of numeric types + * @param average Whether to average the middle two values for even-sized result sets of numeric types; see + * {@link AggSpecMedian#averageEvenlyDivided()} * @param pairs The input/output column name pairs * @return The aggregation */ @@ -342,10 +364,10 @@ static Aggregation AggMin(String... pairs) { } /** - * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied percentile + * Create a {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied percentile * and column name pairs. * - * @param percentile The percentile to use for all component aggregations + * @param percentile The {@link AggSpecPercentile#percentile() percentile} to use for all component aggregations * @param pairs The input/output column name pairs * @return The aggregation */ @@ -354,12 +376,13 @@ static Aggregation AggPct(double percentile, String... pairs) { } /** - * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied percentile + * Create a {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied percentile * and column name pairs. If the percentile equally divides the value space, the result will be the average of the - * values immediately below an above if {@code average} is {@code true}. + * values immediately below and above if {@code average} is {@code true}. * - * @param percentile The percentile to use for all component aggregations - * @param average + * @param percentile The {@link AggSpecPercentile#percentile() percentile} to use for all component aggregations + * @param average Whether to average the lower and higher values for evenly divided result sets of numeric types; + * see {@link AggSpecPercentile#averageEvenlyDivided()} * @param pairs The input/output column name pairs * @return The aggregation */ @@ -368,11 +391,12 @@ static Aggregation AggPct(double percentile, boolean average, String... pairs) { } /** - * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied input - * column name and percentile/output column name pairs. + * Create a {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied input column + * name and percentile/output column name pairs. * * @param inputColumn The input column name - * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @param percentileOutputs The {@link PercentileOutput percentile/output column name pairs} for the component + * aggregations; see {@link #PctOut(double, String)}. * @return The aggregation */ static Aggregation AggPct(String inputColumn, PercentileOutput... percentileOutputs) { @@ -382,12 +406,15 @@ static Aggregation AggPct(String inputColumn, PercentileOutput... percentileOutp } /** - * Create an {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied input - * column name and percentile/output column name pairs. If the percentile equally divides the value space, the - * result will be the average of the values immediately below an above if {@code average} is {@code true}. + * Create a {@link io.deephaven.api.agg.spec.AggSpecPercentile percentile} aggregation for the supplied input column + * name and percentile/output column name pairs. If the percentile equally divides the value space, the result will + * be the average of the values immediately below and above if {@code average} is {@code true}. * * @param inputColumn The input column name - * @param percentileOutputs The percentile/output column name pairs for the component aggregations + * @param average Whether to average the lower and higher values for evenly divided result sets of numeric types; + * see {@link AggSpecPercentile#averageEvenlyDivided()} + * @param percentileOutputs The {@link PercentileOutput percentile/output column name pairs} for the component + * aggregations; see {@link #PctOut(double, String)}. * @return The aggregation */ static Aggregation AggPct(String inputColumn, boolean average, PercentileOutput... percentileOutputs) { @@ -396,34 +423,95 @@ static Aggregation AggPct(String inputColumn, boolean average, PercentileOutput. return of(aggFactory, inputColumn, percentileOutputs); } - static Aggregation AggSortedFirst(String sortedColumn, String... pairs) { - return of(AggSpec.sortedFirst(sortedColumn), pairs); + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecSortedFirst sorted first} aggregation for the supplied sort + * column name and input/output column name pairs. + * + * @param sortColumn The sort column name + * @param pairs The input/output column name pairs for the component aggregations + * @return The aggregation + */ + static Aggregation AggSortedFirst(String sortColumn, String... pairs) { + return of(AggSpec.sortedFirst(sortColumn), pairs); } - static Aggregation AggSortedFirst(Collection sortedColumns, String... pairs) { - return of(AggSpec.sortedFirst(sortedColumns), pairs); + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecSortedFirst sorted first} aggregation for the supplied sort + * column names and input/output column name pairs. + * + * @param sortColumns The sort column names + * @param pairs The input/output column name pairs for the component aggregations + * @return The aggregation + */ + static Aggregation AggSortedFirst(Collection sortColumns, String... pairs) { + return of(AggSpec.sortedFirst(sortColumns), pairs); } - static Aggregation AggSortedLast(String sortedColumn, String... pairs) { - return of(AggSpec.sortedLast(sortedColumn), pairs); + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecSortedLast sorted last} aggregation for the supplied sort column + * name and input/output column name pairs. + * + * @param sortColumn The sort column name + * @param pairs The input/output column name pairs for the component aggregations + * @return The aggregation + */ + static Aggregation AggSortedLast(String sortColumn, String... pairs) { + return of(AggSpec.sortedLast(sortColumn), pairs); } - static Aggregation AggSortedLast(Collection sortedColumns, String... pairs) { - return of(AggSpec.sortedLast(sortedColumns), pairs); + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecSortedLast sorted last} aggregation for the supplied sort column + * names and input/output column name pairs. + * + * @param sortColumns The sort column names + * @param pairs The input/output column name pairs for the component aggregations + * @return The aggregation + */ + static Aggregation AggSortedLast(Collection sortColumns, String... pairs) { + return of(AggSpec.sortedLast(sortColumns), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecStd standard deviation} aggregation for the supplied column name + * pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggStd(String... pairs) { return of(AggSpec.std(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecSum sum} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggSum(String... pairs) { return of(AggSpec.sum(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecTDigest T-Digest} aggregation for the supplied column name pairs + * with the default T-Digest {@link AggSpecTDigest#compression() compression} factor. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggTDigest(String... pairs) { return of(AggSpec.tDigest(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecTDigest T-Digest} aggregation for the supplied column name pairs + * with the supplied {@code compression} factor. + * + * @param compression T-Digest {@link AggSpecTDigest#compression() compression} factor; must be > 1, should + * probably be < 1000 + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggTDigest(double compression, String... pairs) { return of(AggSpec.tDigest(compression), pairs); } @@ -445,7 +533,8 @@ static Aggregation AggUnique(String... pairs) { * This will consider {@code null} values when determining if a group has a single unique value if * {@code includeNulls} is {@code true}. Non-unique groups will have {@code null} values in the output column. * - * @param includeNulls Whether to consider {@code null} values towards uniqueness + * @param includeNulls Whether to consider {@code null} values towards uniqueness; see + * {@link AggSpecUnique#includeNulls()} * @param pairs The input/output column name pairs * @return The aggregation */ @@ -468,18 +557,52 @@ static Aggregation AggUnique(boolean includeNulls, Sentinel nonUniqueSentinel, S return of(AggSpec.unique(includeNulls, nonUniqueSentinel.value()), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecVar variance} aggregation for the supplied column name pairs. + * + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggVar(String... pairs) { return of(AggSpec.var(), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecWAvg weighted average} aggregation for the supplied weight + * column name and column name pairs. + * + * @param weightColumn The {@link AggSpecWAvg#weight() weight column name} + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggWAvg(String weightColumn, String... pairs) { return of(AggSpec.wavg(weightColumn), pairs); } + /** + * Create a {@link io.deephaven.api.agg.spec.AggSpecWSum weighted sum} aggregation for the supplied weight column + * name and column name pairs. + * + * @param weightColumn The {@link AggSpecWSum#weight() weight column name} + * @param pairs The input/output column name pairs + * @return The aggregation + */ static Aggregation AggWSum(String weightColumn, String... pairs) { return of(AggSpec.wsum(weightColumn), pairs); } + /** + * Make a {@link PercentileOutput percentile/output column name pair}. This allows for strongly-typed input to + * various approximate percentile and percentile aggregation factory methods. + * + * @param percentile The percentile for the aggregation + * @param outputColumn The output column name to associate with the percentile + * @return The percentile/output column name pair + * @see #AggApproxPct(String, PercentileOutput...) + * @see #AggApproxPct(String, double, PercentileOutput...) + * @see #AggPct(String, PercentileOutput...) + * @see #AggPct(String, boolean, PercentileOutput...) + */ static PercentileOutput PctOut(double percentile, String outputColumn) { return PercentileOutput.of(percentile, outputColumn); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 541927938da..991c2d2a781 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -83,10 +83,10 @@ static AggSpecFormula formula(String formula) { } /** - * @return {@link AggSpecFormula#of(String, String)} for {@code formula} and {@code formulaParam} + * @return {@link AggSpecFormula#of(String, String)} for {@code formula} and {@code paramToken} */ - static AggSpecFormula formula(String formula, String formulaParam) { - return AggSpecFormula.of(formula, formulaParam); + static AggSpecFormula formula(String formula, String paramToken) { + return AggSpecFormula.of(formula, paramToken); } /** @@ -125,10 +125,10 @@ static AggSpecMedian median() { } /** - * @return {@link AggSpecMedian#of(boolean)} for {@code averageMedian} + * @return {@link AggSpecMedian#of(boolean)} for {@code averageEvenlyDivided} */ - static AggSpecMedian median(boolean averageMedian) { - return AggSpecMedian.of(averageMedian); + static AggSpecMedian median(boolean averageEvenlyDivided) { + return AggSpecMedian.of(averageEvenlyDivided); } /** @@ -146,10 +146,10 @@ static AggSpecPercentile percentile(double percentile) { } /** - * @return {@link AggSpecPercentile#of(double, boolean)} for {@code percentile} and {@code averageMedian} + * @return {@link AggSpecPercentile#of(double, boolean)} for {@code percentile} and {@code averageEvenlyDivided} */ - static AggSpecPercentile percentile(double percentile, boolean averageMedian) { - return AggSpecPercentile.of(percentile, averageMedian); + static AggSpecPercentile percentile(double percentile, boolean averageEvenlyDivided) { + return AggSpecPercentile.of(percentile, averageEvenlyDivided); } /** diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java index 70f02377676..215f82b3715 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAbsSum.java @@ -1,10 +1,21 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; /** - * Specifies an aggregation that supplies the absolute sum for each group. Only works with numeric input types. + * Specifies an aggregation that outputs the sum of absolute input values for each group. Only works with numeric input + * types and {@link Boolean}. + *

+ * {@link Boolean} inputs are aggregated according to the following rules: + *

    + *
  • If any input value is {@code true}, the output value is {@code true}
  • + *
  • If there are no non-{@code null} input values, the output value is {@code null}
  • + *
  • Else all input values must be {@code false}, and the output value is {@code false}
  • + *
+ * + * @see TableOperations#absSumBy */ @Immutable @SimpleStyle diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java index b66c6dde6bf..7593ecb9a3f 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.java @@ -7,8 +7,14 @@ import org.immutables.value.Value.Parameter; /** - * Approximate percentile aggregation using a T-Digest for calculation. Efficiently supports multiple output percentiles - * based on a single input column. May only be used on static or add-only tables. + * Specifies an aggregation that outputs a percentile approximated using a T-Digest with the specified + * {@link #compression()}. + *

+ * Efficiently supports multiple output percentiles based on a single input column. + *

+ * May only be used on static or add-only tables. + *

+ * Only supported for numeric types. */ @Immutable @BuildableStyle diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java index ba6737a97bf..6a73ded2aae 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecAvg.java @@ -3,6 +3,11 @@ import io.deephaven.annotations.SimpleStyle; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the arithmetic mean for each group. Only works with numeric input types. + * + * @see io.deephaven.api.TableOperations#avgBy + */ @Immutable @SimpleStyle public abstract class AggSpecAvg extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java index 45a61621f4d..7efbc3ff751 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecCountDistinct.java @@ -4,6 +4,9 @@ import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the count of distinct values for each group. + */ @Immutable @BuildableStyle public abstract class AggSpecCountDistinct extends AggSpecBase { @@ -21,6 +24,11 @@ public final String description() { return "count distinct" + (countNulls() ? " (counting nulls)" : ""); } + /** + * Whether {@code null} input values should be included when counting the distinct input values. + * + * @return Whether to count nulls + */ @Default public boolean countNulls() { return false; diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java index 388306ccbd1..d51a84ebdf4 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecDistinct.java @@ -4,6 +4,10 @@ import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the distinct values for each group as a Deephaven vector + * (io.deephaven.vector.Vector). + */ @Immutable @BuildableStyle public abstract class AggSpecDistinct extends AggSpecBase { @@ -21,6 +25,11 @@ public final String description() { return "distinct" + (includeNulls() ? " (including nulls)" : ""); } + /** + * Whether {@code null} input values should be included in the distinct output values. + * + * @return Whether to include nulls + */ @Default public boolean includeNulls() { return false; diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java index 49b6a3372e7..19ecabf1cb4 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFirst.java @@ -1,8 +1,14 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the first value in the input column for each group. + * + * @see TableOperations#firstBy + */ @Immutable @SimpleStyle public abstract class AggSpecFirst extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java index 62db0da597e..f40325d075f 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java @@ -1,14 +1,14 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.BuildableStyle; -import io.deephaven.annotations.SimpleStyle; -import io.deephaven.api.expression.Expression; import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; -import org.immutables.value.Value.Parameter; - -import java.util.stream.Collectors; +/** + * Specifies an aggregation that applies a {@link #formula() formula} to each input group (as a Deephaven vector + * (io.deephaven.vector.Vector)) to produce the corresponding output value. Each input column name is substituted for + * the {@link #paramToken() param token} for evaluation. + */ @Immutable @BuildableStyle public abstract class AggSpecFormula extends AggSpecBase { @@ -17,19 +17,29 @@ public static AggSpecFormula of(String formula) { return ImmutableAggSpecFormula.builder().formula(formula).build(); } - public static AggSpecFormula of(String formula, String formulaParam) { - return ImmutableAggSpecFormula.builder().formula(formula).formulaParam(formulaParam).build(); + public static AggSpecFormula of(String formula, String paramToken) { + return ImmutableAggSpecFormula.builder().formula(formula).paramToken(paramToken).build(); } @Override public final String description() { - return "formula '" + formula() + "' with column param '" + formulaParam() + '\''; + return "formula '" + formula() + "' with column param '" + paramToken() + '\''; } + /** + * The formula to use to calculate output values from grouped input values. + * + * @return The formula + */ public abstract String formula(); + /** + * The formula parameter token to be replaced with the input column name for evaluation. + * + * @return The parameter token + */ @Default - public String formulaParam() { + public String paramToken() { return "each"; } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java index a3d21d828be..ed60a0c07c0 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFreeze.java @@ -4,12 +4,12 @@ import org.immutables.value.Value.Immutable; /** - * Aggregation that freezes the first value for each group and ignores subsequent changes. When groups are removed, the - * corresponding output row is removed. When groups are re-added (on a subsequent update cycle), the newly added value - * is then frozen. - *

- * Only one row per group is allowed in the output, because the operation has no way to determine which row to freeze - * otherwise. This is a constraint on the input data. + * Specifies an aggregation that freezes the first value for each group and ignores subsequent changes. When groups are + * removed, the corresponding output row is removed. When groups are re-added (on a subsequent update cycle), the newly + * added value is then frozen. + * + * @implNote Only one row per group is allowed in the output, because the operation has no way to determine which row to + * freeze otherwise. This is a constraint on the input data. */ @Immutable @SimpleStyle diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java index ea09f9d670f..fa20e16e578 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecGroup.java @@ -3,6 +3,13 @@ import io.deephaven.annotations.SimpleStyle; import org.immutables.value.Value.Immutable; +import java.util.Collection; + +/** + * Specifies an aggregation that outputs each group of input values as a Deephaven vector (io.deephaven.vector.Vector). + * + * @see io.deephaven.api.TableOperations#groupBy + */ @Immutable @SimpleStyle public abstract class AggSpecGroup extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java index 7704c140a45..69d07a83041 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecLast.java @@ -1,8 +1,14 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the last value in the input column for each group. + * + * @see TableOperations#lastBy + */ @Immutable @SimpleStyle public abstract class AggSpecLast extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java index 820d02cfa6d..73334754ce4 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMax.java @@ -1,8 +1,15 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the maximum value in the input column for each group. Only works for numeric or + * {@link Comparable} input types. + * + * @see TableOperations#maxBy + */ @Immutable @SimpleStyle public abstract class AggSpecMax extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java index 378995e287a..7259fadced7 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMedian.java @@ -4,6 +4,10 @@ import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +/** + * Specifier for a column aggregation that produces a median value from the input column's values for each group. Only + * works for numeric or {@link Comparable} input types. + */ @Immutable @BuildableStyle public abstract class AggSpecMedian extends AggSpecBase { @@ -12,17 +16,23 @@ public static AggSpecMedian of() { return ImmutableAggSpecMedian.builder().build(); } - public static AggSpecMedian of(boolean averageMedian) { - return ImmutableAggSpecMedian.builder().averageMedian(averageMedian).build(); + public static AggSpecMedian of(boolean averageEvenlyDivided) { + return ImmutableAggSpecMedian.builder().averageEvenlyDivided(averageEvenlyDivided).build(); } @Override public final String description() { - return "median" + (averageMedian() ? " averaging median" : ""); + return "median" + (averageEvenlyDivided() ? " (averaging when evenly divided)" : ""); } + /** + * Whether to average the highest low-bucket value and lowest high-bucket value, when the low-bucket and high-bucket + * are of equal size. Only applies to numeric types. + * + * @return Whether to average the two result candidates for evenly-divided input sets of numeric types + */ @Default - public boolean averageMedian() { + public boolean averageEvenlyDivided() { return true; } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java index 1146bc297e3..64b346691ad 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecMin.java @@ -1,8 +1,15 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the minimum value in the input column for each group. Only works for numeric or + * {@link Comparable} input types. + * + * @see TableOperations#minBy + */ @Immutable @SimpleStyle public abstract class AggSpecMin extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java index 1bc3068a75a..9bf74dca94a 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecPercentile.java @@ -5,6 +5,10 @@ import org.immutables.value.Value.Default; import org.immutables.value.Value.Immutable; +/** + * Specifier for a column aggregation that produces a percentile value from the input column's values for each group. + * Only works for numeric or {@link Comparable} input types. + */ @Immutable @BuildableStyle public abstract class AggSpecPercentile extends AggSpecBase { @@ -13,19 +17,33 @@ public static AggSpecPercentile of(double percentile) { return ImmutableAggSpecPercentile.builder().percentile(percentile).build(); } - public static AggSpecPercentile of(double percentile, boolean averageMedian) { - return ImmutableAggSpecPercentile.builder().percentile(percentile).averageMedian(averageMedian).build(); + public static AggSpecPercentile of(double percentile, boolean averageEvenlyDivided) { + return ImmutableAggSpecPercentile.builder().percentile(percentile).averageEvenlyDivided(averageEvenlyDivided) + .build(); } @Override public final String description() { - return String.format("%.2f percentile%s", percentile(), averageMedian() ? " averaging median" : ""); + return String.format("%.2f percentile%s", + percentile(), + averageEvenlyDivided() ? " (averaging when evenly divided)" : ""); } + /** + * The percentile to calculate. Must be >= 0.0 and <= 1.0. + * + * @return The percentile to calculate + */ public abstract double percentile(); + /** + * Whether to average the highest low-bucket value and lowest high-bucket value, when the low-bucket and high-bucket + * are of equal size. Only applies to numeric types. + * + * @return Whether to average the two result candidates for evenly-divided input sets of numeric types + */ @Default - public boolean averageMedian() { + public boolean averageEvenlyDivided() { return false; } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java index a72d865b9ab..bfe259f0d92 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedFirst.java @@ -9,8 +9,10 @@ import java.util.stream.Collectors; /** - * Note: the sorted-first aggregation only supports {@link SortColumn.Order#ASCENDING} columns at the moment. - * + * Specifies an aggregation that outputs the first value in the input column for each group, after sorting the group on + * the {@link #columns() sort columns}. + * + * @implNote The sorted-first aggregation only supports {@link SortColumn.Order#ASCENDING} columns at the moment. * @see SortedFirst / SortedLast aggregations with sort * direction */ @@ -27,6 +29,11 @@ public final String description() { return "first sorted by " + columns().stream().map(sc -> sc.column().name()).collect(Collectors.joining(", ")); } + /** + * The columns to sort on to determine the order within each group. + * + * @return The sort columns + */ public abstract List columns(); @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java index fc25de13c92..b5712560626 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSortedLast.java @@ -9,8 +9,10 @@ import java.util.stream.Collectors; /** - * Note: the sorted-last aggregation only supports {@link SortColumn.Order#ASCENDING} columns at the moment. + * Specifies an aggregation that outputs the last value in the input column for each group, after sorting the group on + * the {@link #columns() sort columns}. * + * @implNote The sorted-last aggregation only supports {@link SortColumn.Order#ASCENDING} columns at the moment. * @see SortedFirst / SortedLast aggregations with sort * direction */ @@ -27,6 +29,11 @@ public final String description() { return "last sorted by " + columns().stream().map(sc -> sc.column().name()).collect(Collectors.joining(", ")); } + /** + * The columns to sort on to determine the order within each group. + * + * @return The sort columns + */ public abstract List columns(); @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java index be089079538..96be57b7d76 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecStd.java @@ -1,8 +1,15 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the standard deviation of the input column values for each group. Only works + * for numeric input types. + * + * @see TableOperations#stdBy + */ @Immutable @SimpleStyle public abstract class AggSpecStd extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java index 40e366cadca..d045b30c727 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecSum.java @@ -1,8 +1,22 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the sum of input values for each group. Only works with numeric input types and + * {@link Boolean}. + *

+ * {@link Boolean} inputs are aggregated according to the following rules: + *

    + *
  • If any input value is {@code true}, the output value is {@code true}
  • + *
  • If there are no non-{@code null} input values, the output value is {@code null}
  • + *
  • Else all input values must be {@code false}, and the output value is {@code false}
  • + *
+ * + * @see TableOperations#sumBy + */ @Immutable @SimpleStyle public abstract class AggSpecSum extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java index 0d34bd7c3d8..a293cfab12a 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecTDigest.java @@ -6,8 +6,8 @@ import org.immutables.value.Value.Immutable; /** - *

- * Aggregates input column into a TDigest output column with the specified compression. + * Specifies an aggregation that outputs a T-Digest (com.tdunning.math.stats.TDigest) with the specified + * {@link #compression()}. *

* May be used to implement parallel percentile calculations by splitting inputs and accumulating results into a single * downstream TDigest. diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java index 96689529e11..f4eeb2e7419 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecUnique.java @@ -6,6 +6,10 @@ import javax.annotation.Nullable; +/** + * Specifies an aggregation that outputs the single unique input value for groups that have one, {@code null} if all + * input values are {@code null}, or {@link #nonUniqueSentinel()} if there is more than one distinct value. + */ @Immutable @BuildableStyle public abstract class AggSpecUnique extends AggSpecBase { @@ -43,11 +47,22 @@ public final String description() { return "unique" + (includeNulls() ? " (including nulls)" : ""); } + /** + * Whether to include {@code null} values as a distinct value for determining if there is only one unique value to + * output. + * + * @return Whether to include nulls + */ @Default public boolean includeNulls() { return false; } + /** + * The output value to use for groups that don't have a single unique input value. + * + * @return The non-unique sentinel value + */ @Nullable @Default public Object nonUniqueSentinel() { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java index 4a2a95b54a2..db6725cd8de 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecVar.java @@ -1,8 +1,15 @@ package io.deephaven.api.agg.spec; import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; +/** + * Specifies an aggregation that outputs the variance of the input column values for each group. Only works for numeric + * input types. + * + * @see TableOperations#varBy + */ @Immutable @SimpleStyle public abstract class AggSpecVar extends AggSpecEmptyBase { diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java index 155c0dd88ef..80e6b8c8bbd 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWAvg.java @@ -2,9 +2,16 @@ import io.deephaven.annotations.SimpleStyle; import io.deephaven.api.ColumnName; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Parameter; +/** + * Specifies an aggregation that outputs the average (arithmetic mean) of the input column values weighted by the + * {@link #weight() weight column} values for each group. Only works for numeric input types. + * + * @see TableOperations#wavgBy + */ @Immutable @SimpleStyle public abstract class AggSpecWAvg extends AggSpecBase { @@ -18,6 +25,11 @@ public final String description() { return "average weighted by " + weight(); } + /** + * Column name for the source of input weights. + * + * @return The weight column name + */ @Parameter public abstract ColumnName weight(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java index acf010dd9a9..4aaeadabfc7 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecWSum.java @@ -2,9 +2,16 @@ import io.deephaven.annotations.SimpleStyle; import io.deephaven.api.ColumnName; +import io.deephaven.api.TableOperations; import org.immutables.value.Value.Immutable; import org.immutables.value.Value.Parameter; +/** + * Specifies an aggregation that outputs the sum of the input column values weighted by the {@link #weight() weight + * column} values for each group. Only works for numeric input types. + * + * @see TableOperations#wsumBy + */ @Immutable @SimpleStyle public abstract class AggSpecWSum extends AggSpecBase { @@ -18,6 +25,11 @@ public final String description() { return "sum weighted by " + weight(); } + /** + * Column name for the source of input weights. + * + * @return The weight column name + */ @Parameter public abstract ColumnName weight(); From ed5dffbaecd8d7e1a7bbff8814e6afad86e97526 Mon Sep 17 00:00:00 2001 From: Ryan Caudy Date: Thu, 3 Feb 2022 12:44:20 -0500 Subject: [PATCH 44/44] Spotless, replication, and docs --- .../doc/io/deephaven/api/agg/Aggregation.json | 59 ++++++++++--------- .../api/agg/Aggregation/Visitor.json | 3 +- .../api/agg/AggregationDescriptions.json | 2 +- .../api/agg/AggregationOptimizer.json | 2 +- .../deephaven/api/agg/AggregationOutputs.json | 7 --- .../deephaven/api/agg/AggregationPairs.json | 2 +- .../io/deephaven/api/agg/Aggregations.json | 2 +- .../deephaven/api/agg/ColumnAggregation.json | 4 +- .../deephaven/api/agg/ColumnAggregations.json | 4 +- .../doc/io/deephaven/api/agg/Count.json | 3 +- .../doc/io/deephaven/api/agg/FirstRowKey.json | 3 +- .../doc/io/deephaven/api/agg/LastRowKey.json | 3 +- .../io/deephaven/api/agg/spec/AggSpec.json | 54 ++++++++--------- .../api/agg/spec/AggSpec/Visitor.json | 2 +- .../deephaven/api/agg/spec/AggSpecAbsSum.json | 5 +- .../spec/AggSpecApproximatePercentile.json | 6 +- .../io/deephaven/api/agg/spec/AggSpecAvg.json | 5 +- .../deephaven/api/agg/spec/AggSpecBase.json | 2 +- .../api/agg/spec/AggSpecColumnReferences.json | 2 +- .../api/agg/spec/AggSpecCountDistinct.json | 7 ++- .../api/agg/spec/AggSpecDistinct.json | 7 ++- .../deephaven/api/agg/spec/AggSpecFirst.json | 5 +- .../api/agg/spec/AggSpecFormula.json | 11 ++-- .../deephaven/api/agg/spec/AggSpecFreeze.json | 6 +- .../deephaven/api/agg/spec/AggSpecGroup.json | 5 +- .../deephaven/api/agg/spec/AggSpecLast.json | 5 +- .../io/deephaven/api/agg/spec/AggSpecMax.json | 5 +- .../deephaven/api/agg/spec/AggSpecMedian.json | 9 +-- .../io/deephaven/api/agg/spec/AggSpecMin.json | 5 +- .../api/agg/spec/AggSpecPercentile.json | 11 ++-- .../api/agg/spec/AggSpecSortedFirst.json | 8 +-- .../api/agg/spec/AggSpecSortedLast.json | 8 +-- .../io/deephaven/api/agg/spec/AggSpecStd.json | 5 +- .../io/deephaven/api/agg/spec/AggSpecSum.json | 5 +- .../api/agg/spec/AggSpecTDigest.json | 6 +- .../deephaven/api/agg/spec/AggSpecUnique.json | 9 +-- .../io/deephaven/api/agg/spec/AggSpecVar.json | 5 +- .../deephaven/api/agg/spec/AggSpecWAvg.json | 7 ++- .../deephaven/api/agg/spec/AggSpecWSum.json | 7 ++- .../impl/by/rollup/RollupAggregation.json | 2 +- .../impl/by/rollup/RollupAggregationBase.json | 2 +- .../table/impl/by/AggregationProcessor.java | 3 +- .../io/deephaven/api/agg/Aggregation.java | 4 +- .../io/deephaven/api/agg/spec/AggSpec.java | 4 +- 44 files changed, 169 insertions(+), 152 deletions(-) delete mode 100644 Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json index 5c0a3fa276b..1b193a41c97 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation.json @@ -1,35 +1,36 @@ { "className": "io.deephaven.api.agg.Aggregation", "methods": { - "AggAbsSum": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggApproxPct": "*Overload 1* \n :param percentile: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param inputColumn: java.lang.String\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n :param inputColumn: java.lang.String\n :param compression: double\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation", - "AggAvg": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggCount": ":param resultColumn: java.lang.String\n:return: io.deephaven.api.agg.Aggregation", - "AggCountDistinct": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param countNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggDistinct": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggFirst": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggFirstRowKey": ":param resultColumn: java.lang.String\n:return: io.deephaven.api.agg.Aggregation", - "AggFormula": ":param formula: java.lang.String\n:param formulaParam: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggGroup": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggLast": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggLastRowKey": ":param resultColumn: java.lang.String\n:return: io.deephaven.api.agg.Aggregation", - "AggMax": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggMed": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param average: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggMin": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggPct": "*Overload 1* \n :param percentile: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param percentile: double\n :param average: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param inputColumn: java.lang.String\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n :param inputColumn: java.lang.String\n :param average: boolean\n :param percentileOutputs: io.deephaven.api.agg.util.PercentileOutput...\n :return: io.deephaven.api.agg.Aggregation", - "AggSortedFirst": "*Overload 1* \n :param sortedColumn: java.lang.String\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param sortedColumns: java.util.Collection\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggSortedLast": "*Overload 1* \n :param sortedColumn: java.lang.String\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param sortedColumns: java.util.Collection\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggStd": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggSum": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggTDigest": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param compression: double\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggUnique": "*Overload 1* \n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 2* \n :param includeNulls: boolean\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param includeNulls: boolean\n :param nonUniqueSentinel: io.deephaven.api.agg.util.Sentinel\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation", - "AggVar": ":param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggWAvg": ":param weightColumn: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "AggWSum": ":param weightColumn: java.lang.String\n:param pairs: java.lang.String...\n:return: io.deephaven.api.agg.Aggregation", - "PctOut": ":param percentile: double\n:param outputColumn: java.lang.String\n:return: io.deephaven.api.agg.util.PercentileOutput", - "Sentinel": "*Overload 1* \n :param value: java.lang.Object\n :return: io.deephaven.api.agg.util.Sentinel\n \n*Overload 2* \n :return: io.deephaven.api.agg.util.Sentinel", - "of": "*Overload 1* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pair: java.lang.String\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param spec: io.deephaven.api.agg.spec.AggSpec\n :param pairs: java.lang.String...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param aggregations: io.deephaven.api.agg.Aggregation...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 4* \n Note: Java generics information - \n \n :param columnAggFactory: java.util.function.BiFunction\n :param inputColumn: java.lang.String\n :param inputs: INPUT_TYPE...\n :return: io.deephaven.api.agg.Aggregation", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "AggAbsSum": "Create an absolute sum aggregation for the supplied column name\n pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggApproxPct": "**Incompatible overloads text - text from the first overload:**\n\nCreate an approximate percentile aggregation for\n the supplied input column name and percentile/output column name pairs with the default T-Digest\n compression factor.\n\n*Overload 1* \n :param percentile: (double) - The percentile to use for all component\n aggregations\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param percentile: (double) - The percentile to use for all component\n aggregations\n :param compression: (double) - T-Digest compression factor; must be > 1, should\n probably be < 1000\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 3* \n :param inputColumn: (java.lang.String) - The input column name\n :param percentileOutputs: (io.deephaven.api.agg.util.PercentileOutput...) - The percentile/output column name pairs for the component\n aggregations; see PctOut(double, String).\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 4* \n :param inputColumn: (java.lang.String) - The input column name\n :param compression: (double) - T-Digest compression factor; must be > 1, should\n probably be < 1000\n :param percentileOutputs: (io.deephaven.api.agg.util.PercentileOutput...) - The percentile/output column name pairs for the component\n aggregations; see PctOut(double, String).\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggAvg": "Create an average (arithmetic mean) aggregation for the\n supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggCount": "Create a count aggregation with the supplied output column name.\n\n:param resultColumn: (java.lang.String) - The output column name\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggCountDistinct": "**Incompatible overloads text - text from the first overload:**\n\nCreate a count distinct aggregation for the supplied\n column name pairs. This will count null values from the input column(s) if countNulls is\n true.\n\n*Overload 1* \n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param countNulls: (boolean) - Whether null values should be counted; see AggSpecCountDistinct.countNulls()}\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggDistinct": "**Incompatible overloads text - text from the first overload:**\n\nCreate a distinct aggregation for the supplied column name\n pairs. This will include null values in the output column(s) if includeNulls is true.\n\n*Overload 1* \n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param includeNulls: (boolean) - Whether null values should be included; see AggSpecDistinct.includeNulls()\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggFirst": "Create a first aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggFirstRowKey": "Create a first row key aggregation with the supplied result column.\n\n:param resultColumn: (java.lang.String) - The output column name\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggFormula": "Create a formula aggregation with the supplied formula,\n paramToken, and column name pairs.\n\n:param formula: (java.lang.String) - The formula to use for all input columns to produce all output\n columns\n:param paramToken: (java.lang.String) - The parameter token to replace with the input column name\n in formula\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggFreeze": "Create a freeze aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggGroup": "Create a group aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggLast": "Create a last aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggLastRowKey": "Create a last row key aggregation with the supplied result column.\n\n:param resultColumn: (java.lang.String) - The output column name\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggMax": "Create a max aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggMed": "**Incompatible overloads text - text from the first overload:**\n\nCreate a median aggregation for the supplied column name pairs.\n For numeric types, if there are an even number of values the result will be an average of the two middle values\n if average is true, else the result will be the lower of the two middle values.\n\n*Overload 1* \n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param average: (boolean) - Whether to average the middle two values for even-sized result sets of numeric types; see\n AggSpecMedian.averageEvenlyDivided()\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggMin": "Create a min aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggPct": "**Incompatible overloads text - text from the first overload:**\n\nCreate a percentile aggregation for the supplied input column\n name and percentile/output column name pairs.\n\n*Overload 1* \n :param percentile: (double) - The percentile to use for all component aggregations\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param percentile: (double) - The percentile to use for all component aggregations\n :param average: (boolean) - Whether to average the lower and higher values for evenly divided result sets of numeric types;\n see AggSpecPercentile.averageEvenlyDivided()\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 3* \n :param inputColumn: (java.lang.String) - The input column name\n :param percentileOutputs: (io.deephaven.api.agg.util.PercentileOutput...) - The percentile/output column name pairs for the component\n aggregations; see PctOut(double, String).\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 4* \n :param inputColumn: (java.lang.String) - The input column name\n :param average: (boolean) - Whether to average the lower and higher values for evenly divided result sets of numeric types;\n see AggSpecPercentile.averageEvenlyDivided()\n :param percentileOutputs: (io.deephaven.api.agg.util.PercentileOutput...) - The percentile/output column name pairs for the component\n aggregations; see PctOut(double, String).\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggSortedFirst": "**Incompatible overloads text - text from the first overload:**\n\nCreate a sorted first aggregation for the supplied sort\n column name and input/output column name pairs.\n\n*Overload 1* \n :param sortColumn: (java.lang.String) - The sort column name\n :param pairs: (java.lang.String...) - The input/output column name pairs for the component aggregations\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param sortColumns: (java.util.Collection) - The sort column names\n :param pairs: (java.lang.String...) - The input/output column name pairs for the component aggregations\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggSortedLast": "**Incompatible overloads text - text from the first overload:**\n\nCreate a sorted last aggregation for the supplied sort column\n name and input/output column name pairs.\n\n*Overload 1* \n :param sortColumn: (java.lang.String) - The sort column name\n :param pairs: (java.lang.String...) - The input/output column name pairs for the component aggregations\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param sortColumns: (java.util.Collection) - The sort column names\n :param pairs: (java.lang.String...) - The input/output column name pairs for the component aggregations\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggStd": "Create a standard deviation aggregation for the supplied column name\n pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggSum": "Create a sum aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggTDigest": "**Incompatible overloads text - text from the first overload:**\n\nCreate a T-Digest aggregation for the supplied column name pairs\n with the default T-Digest compression factor.\n\n*Overload 1* \n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param compression: (double) - T-Digest compression factor; must be > 1, should\n probably be < 1000\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggUnique": "**Incompatible overloads text - text from the first overload:**\n\nCreate a unique aggregation for the supplied column name pairs.\n This will consider null values when determining if a group has a single unique value if\n includeNulls is true. Non-unique groups will have null values in the output column.\n\n*Overload 1* \n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 2* \n :param includeNulls: (boolean) - Whether to consider null values towards uniqueness; see\n AggSpecUnique.includeNulls()\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 3* \n :param includeNulls: (boolean) - Whether to consider null values towards uniqueness\n :param nonUniqueSentinel: (io.deephaven.api.agg.util.Sentinel) - The value to output for non-unique groups\n :param pairs: (java.lang.String...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggVar": "Create a variance aggregation for the supplied column name pairs.\n\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggWAvg": "Create a weighted average aggregation for the supplied weight\n column name and column name pairs.\n\n:param weightColumn: (java.lang.String) - The weight column name\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "AggWSum": "Create a weighted sum aggregation for the supplied weight column\n name and column name pairs.\n\n:param weightColumn: (java.lang.String) - The weight column name\n:param pairs: (java.lang.String...) - The input/output column name pairs\n:return: (io.deephaven.api.agg.Aggregation) The aggregation", + "PctOut": "Make a percentile/output column name pair. This allows for strongly-typed input to\n various approximate percentile and percentile aggregation factory methods.\n\n:param percentile: (double) - The percentile for the aggregation\n:param outputColumn: (java.lang.String) - The output column name to associate with the percentile\n:return: (io.deephaven.api.agg.util.PercentileOutput) The percentile/output column name pair", + "Sentinel": "**Incompatible overloads text - text from the first overload:**\n\nMake a sentinel wrapping null. This serves to avoid ambiguity in the var-args overloads\n of some Aggregation factory methods.\n\n*Overload 1* \n :param value: (java.lang.Object) - The value to wrap\n :return: (io.deephaven.api.agg.util.Sentinel) The sentinel\n \n*Overload 2* \n :return: (io.deephaven.api.agg.util.Sentinel) The sentinel", + "of": "**Incompatible overloads text - text from the first overload:**\n\nCombine an AggSpec and an input/output column name pair into a ColumnAggregation.\n\n*Overload 1* \n :param spec: (io.deephaven.api.agg.spec.AggSpec) - The aggregation specifier to apply to the column name pair\n :param pair: (java.lang.String) - The input/output column name pair\n :return: (io.deephaven.api.agg.ColumnAggregation) The aggregation\n \n*Overload 2* \n :param spec: (io.deephaven.api.agg.spec.AggSpec) - The aggregation specifier to apply to the column name pair(s)\n :param pairs: (java.lang.String...) - The input/output column name pair or\n pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 3* \n :param aggregations: (io.deephaven.api.agg.Aggregation...) - The aggregations to combine\n :return: (io.deephaven.api.agg.Aggregation) The combined aggregation\n \n*Overload 4* \n Note: Java generics information - \n \n :param columnAggFactory: (java.util.function.BiFunction) - A factory for combining an input column and input value into a aggregation\n :param inputColumn: (java.lang.String) - The input column for each component of the resulting aggregation\n :param inputs: (INPUT_TYPE...) - The input values to combine with the input column via the factory\n :return: (io.deephaven.api.agg.Aggregation) The combined aggregation", + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.Aggregation", "text": "Represents an aggregation that can be applied to a table.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json index 41f4259e649..98913d66971 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregation/Visitor.json @@ -1,8 +1,9 @@ { "className": "io.deephaven.api.agg.Aggregation$Visitor", "methods": { - "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "**Incompatible overloads text - text from the first overload:**\n\nVisit a column aggregation.\n\n*Overload 1* \n :param aggregations: (io.deephaven.api.agg.Aggregations) - The compound aggregation to visit\n \n*Overload 2* \n :param columnAgg: (io.deephaven.api.agg.ColumnAggregation) - The column aggregation to visit\n \n*Overload 3* \n :param columnAggs: (io.deephaven.api.agg.ColumnAggregations) - The compound column aggregation to visit\n \n*Overload 4* \n :param count: (io.deephaven.api.agg.Count) - The count aggregation\n \n*Overload 5* \n :param firstRowKey: (io.deephaven.api.agg.FirstRowKey) - The first row key aggregation\n \n*Overload 6* \n :param lastRowKey: (io.deephaven.api.agg.LastRowKey) - The last row key aggregation" }, "path": "io.deephaven.api.agg.Aggregation.Visitor", + "text": "Visitor interface. Combines with Aggregation.walk(Visitor) in order to allow for type-safe Aggregation evaluation\n without switch statements or if-else blocks.", "typeName": "interface" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json index 6611c85a363..1416297215e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationDescriptions.json @@ -2,7 +2,7 @@ "className": "io.deephaven.api.agg.AggregationDescriptions", "methods": { "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.Map\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.Map", - "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "**Incompatible overloads text - text from the first overload:**\n\nVisit a column aggregation.\n\n*Overload 1* \n :param aggregations: (io.deephaven.api.agg.Aggregations) - The compound aggregation to visit\n \n*Overload 2* \n :param columnAgg: (io.deephaven.api.agg.ColumnAggregation) - The column aggregation to visit\n \n*Overload 3* \n :param columnAggs: (io.deephaven.api.agg.ColumnAggregations) - The compound column aggregation to visit\n \n*Overload 4* \n :param count: (io.deephaven.api.agg.Count) - The count aggregation\n \n*Overload 5* \n :param firstRowKey: (io.deephaven.api.agg.FirstRowKey) - The first row key aggregation\n \n*Overload 6* \n :param lastRowKey: (io.deephaven.api.agg.LastRowKey) - The last row key aggregation" }, "path": "io.deephaven.api.agg.AggregationDescriptions", "text": "A visitor to describe the input and aggregation column name pairs for aggregations.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json index 23f4ecccb03..0e1ceca72b7 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOptimizer.json @@ -3,7 +3,7 @@ "methods": { "build": ":return: java.util.List", "of": "Optimizes a collection of aggregations by grouping like-specced aggregations together. The\n input order will be preserved based on the spec-encountered order.\n\n:param aggregations: (java.util.Collection) - the aggregations\n:return: (java.util.List) the optimized aggregations", - "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "**Incompatible overloads text - text from the first overload:**\n\nVisit a column aggregation.\n\n*Overload 1* \n :param aggregations: (io.deephaven.api.agg.Aggregations) - The compound aggregation to visit\n \n*Overload 2* \n :param columnAgg: (io.deephaven.api.agg.ColumnAggregation) - The column aggregation to visit\n \n*Overload 3* \n :param columnAggs: (io.deephaven.api.agg.ColumnAggregations) - The compound column aggregation to visit\n \n*Overload 4* \n :param count: (io.deephaven.api.agg.Count) - The count aggregation\n \n*Overload 5* \n :param firstRowKey: (io.deephaven.api.agg.FirstRowKey) - The first row key aggregation\n \n*Overload 6* \n :param lastRowKey: (io.deephaven.api.agg.LastRowKey) - The last row key aggregation" }, "path": "io.deephaven.api.agg.AggregationOptimizer", "text": "Optimizes a collection of aggregations by grouping like-specced aggregations together.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json deleted file mode 100644 index e82296fcf91..00000000000 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationOutputs.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "className": "io.deephaven.api.agg.AggregationOutputs", - "methods": {}, - "path": "io.deephaven.api.agg.AggregationOutputs", - "text": "Helper to get the ordered output column names for aggregations.", - "typeName": "class" -} \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json index 29ce7efcb0c..f17a9d2cfd4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/AggregationPairs.json @@ -3,7 +3,7 @@ "methods": { "of": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", "outputsOf": "*Overload 1* \n :param aggregation: io.deephaven.api.agg.Aggregation\n :return: java.util.stream.Stream\n \n*Overload 2* \n :param aggregations: java.util.Collection\n :return: java.util.stream.Stream", - "visit": "*Overload 1* \n :param aggregations: io.deephaven.api.agg.Aggregations\n \n*Overload 2* \n :param columnAgg: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 3* \n :param columnAggs: io.deephaven.api.agg.ColumnAggregations\n \n*Overload 4* \n :param count: io.deephaven.api.agg.Count\n \n*Overload 5* \n :param firstRowKey: io.deephaven.api.agg.FirstRowKey\n \n*Overload 6* \n :param lastRowKey: io.deephaven.api.agg.LastRowKey" + "visit": "**Incompatible overloads text - text from the first overload:**\n\nVisit a column aggregation.\n\n*Overload 1* \n :param aggregations: (io.deephaven.api.agg.Aggregations) - The compound aggregation to visit\n \n*Overload 2* \n :param columnAgg: (io.deephaven.api.agg.ColumnAggregation) - The column aggregation to visit\n \n*Overload 3* \n :param columnAggs: (io.deephaven.api.agg.ColumnAggregations) - The compound column aggregation to visit\n \n*Overload 4* \n :param count: (io.deephaven.api.agg.Count) - The count aggregation\n \n*Overload 5* \n :param firstRowKey: (io.deephaven.api.agg.FirstRowKey) - The first row key aggregation\n \n*Overload 6* \n :param lastRowKey: (io.deephaven.api.agg.LastRowKey) - The last row key aggregation" }, "path": "io.deephaven.api.agg.AggregationPairs", "text": "A visitor to get the ordered input/output column name pairs for aggregations.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json index 0c60b0e4072..e93022a1c58 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Aggregations.json @@ -3,7 +3,7 @@ "methods": { "aggregations": ":return: java.util.List", "builder": ":return: io.deephaven.api.agg.Aggregations.Builder", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.Aggregations", "text": "Aggregations is an Aggregation that is a collection of two or more aggregations.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json index 70b4c32f879..a6b4151827c 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregation.json @@ -4,9 +4,9 @@ "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:param pair: io.deephaven.api.agg.Pair\n:return: io.deephaven.api.agg.ColumnAggregation", "pair": ":return: io.deephaven.api.agg.Pair", "spec": ":return: io.deephaven.api.agg.spec.AggSpec", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.ColumnAggregation", - "text": "A ColumnAggregation is an Aggregation that is composed of a spec and a single input/output\n column pair.", + "text": "A ColumnAggregation is an Aggregation that is composed of a spec and a single input/output\n column pair. The spec defines the aggregation operation to apply to the input column in order to\n produce the paired output column.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json index a67af9562c9..127e2f472cb 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/ColumnAggregations.json @@ -4,9 +4,9 @@ "builder": ":return: io.deephaven.api.agg.ColumnAggregations.Builder", "pairs": ":return: java.util.List", "spec": ":return: io.deephaven.api.agg.spec.AggSpec", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.ColumnAggregations", - "text": "ColumnAggregations is an Aggregation that is composed of a spec and multiple input/output\n column pairs.", + "text": "ColumnAggregations is an Aggregation that is composed of a spec and multiple input/output\n column pairs. The spec defines the aggregation operation to apply to each input column in order to\n produce the paired output column.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Count.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Count.json index c8a41e86a52..8b36647fb1e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/Count.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/Count.json @@ -3,8 +3,9 @@ "methods": { "column": ":return: io.deephaven.api.ColumnName", "of": "*Overload 1* \n :param name: io.deephaven.api.ColumnName\n :return: io.deephaven.api.agg.Count\n \n*Overload 2* \n :param x: java.lang.String\n :return: io.deephaven.api.agg.Count", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.Count", + "text": "An aggregation that provides a single output column with the number of rows in each aggregation\n group.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json index 9121e5c11aa..8ad168a3cf8 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/FirstRowKey.json @@ -3,8 +3,9 @@ "methods": { "column": ":return: io.deephaven.api.ColumnName", "of": "*Overload 1* \n :param name: io.deephaven.api.ColumnName\n :return: io.deephaven.api.agg.FirstRowKey\n \n*Overload 2* \n :param x: java.lang.String\n :return: io.deephaven.api.agg.FirstRowKey", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.FirstRowKey", + "text": "An aggregation that provides a single output column with the first row key from the input table\n for each aggregation group in the result.\n \n The primary use case for this aggregation is to allow for a subsequent sort on the output column to order aggregated data by current first occurrence in the input table rather than\n encounter order.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json index 8ab4d873e72..7e2037bdb51 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/LastRowKey.json @@ -3,8 +3,9 @@ "methods": { "column": ":return: io.deephaven.api.ColumnName", "of": "*Overload 1* \n :param name: io.deephaven.api.ColumnName\n :return: io.deephaven.api.agg.LastRowKey\n \n*Overload 2* \n :param x: java.lang.String\n :return: io.deephaven.api.agg.LastRowKey", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.LastRowKey", + "text": "An aggregation that provides a single output column with the last row key from the input table\n for each aggregation group in the result.\n \n The primary use case for this aggregation is to allow for a subsequent sort on the output column to order aggregated data by current last occurrence in the input table rather than\n encounter order.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json index d61af47afb5..8082c1b29f4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec.json @@ -1,34 +1,34 @@ { "className": "io.deephaven.api.agg.spec.AggSpec", "methods": { - "absSum": ":return: io.deephaven.api.agg.spec.AggSpecAbsSum", - "aggregation": "*Overload 1* \n :param pair: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param pairs: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param pairs: java.util.Collection\n :return: io.deephaven.api.agg.Aggregation", - "approximatePercentile": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile", - "avg": ":return: io.deephaven.api.agg.spec.AggSpecAvg", - "countDistinct": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 2* \n :param countNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct", - "description": ":return: java.lang.String", - "distinct": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecDistinct", - "first": ":return: io.deephaven.api.agg.spec.AggSpecFirst", - "formula": "*Overload 1* \n :param formula: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 2* \n :param formula: java.lang.String\n :param formulaParam: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula", - "freeze": ":return: io.deephaven.api.agg.spec.AggSpecFreeze", - "group": ":return: io.deephaven.api.agg.spec.AggSpecGroup", - "last": ":return: io.deephaven.api.agg.spec.AggSpecLast", - "max": ":return: io.deephaven.api.agg.spec.AggSpecMax", - "median": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 2* \n :param averageMedian: boolean\n :return: io.deephaven.api.agg.spec.AggSpecMedian", - "min": ":return: io.deephaven.api.agg.spec.AggSpecMin", - "percentile": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 2* \n :param percentile: double\n :param averageMedian: boolean\n :return: io.deephaven.api.agg.spec.AggSpecPercentile", - "sortedFirst": "*Overload 1* \n :param columns: java.lang.String...\n :return: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 2* \n :param columns: java.util.Collection\n :return: io.deephaven.api.agg.spec.AggSpecSortedFirst", - "sortedLast": "*Overload 1* \n :param columns: java.lang.String...\n :return: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 2* \n :param columns: java.util.Collection\n :return: io.deephaven.api.agg.spec.AggSpecSortedLast", - "std": ":return: io.deephaven.api.agg.spec.AggSpecStd", - "sum": ":return: io.deephaven.api.agg.spec.AggSpecSum", - "tDigest": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 2* \n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecTDigest", - "unique": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 2* \n :param includeNulls: boolean\n :param nonUniqueSentinel: java.lang.Object\n :return: io.deephaven.api.agg.spec.AggSpecUnique", - "var": ":return: io.deephaven.api.agg.spec.AggSpecVar", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V", - "wavg": ":param weightColumn: java.lang.String\n:return: io.deephaven.api.agg.spec.AggSpecWAvg", - "wsum": ":param weightColumn: java.lang.String\n:return: io.deephaven.api.agg.spec.AggSpecWSum" + "absSum": ":return: (io.deephaven.api.agg.spec.AggSpecAbsSum) AggSpecAbsSum.of()", + "aggregation": "**Incompatible overloads text - text from the first overload:**\n\nBuild a ColumnAggregation for this AggSpec.\n\n*Overload 1* \n :param pair: (io.deephaven.api.agg.Pair) - The input/output column name pair\n :return: (io.deephaven.api.agg.ColumnAggregation) The aggregation\n \n*Overload 2* \n :param pairs: (io.deephaven.api.agg.Pair...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 3* \n :param pairs: (java.util.Collection) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation", + "approximatePercentile": "*Overload 1* \n :param percentile: double\n :return: (io.deephaven.api.agg.spec.AggSpecApproximatePercentile) AggSpecApproximatePercentile.of(double) for percentile\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :return: (io.deephaven.api.agg.spec.AggSpecApproximatePercentile) AggSpecApproximatePercentile.of(double, double) for percentile and compression", + "avg": ":return: (io.deephaven.api.agg.spec.AggSpecAvg) AggSpecAvg.of()", + "countDistinct": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 2* \n :param countNulls: boolean\n :return: (io.deephaven.api.agg.spec.AggSpecCountDistinct) AggSpecCountDistinct.of(boolean) for countNulls", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "distinct": "*Overload 1* \n :return: (io.deephaven.api.agg.spec.AggSpecDistinct) AggSpecDistinct.of()\n \n*Overload 2* \n :param includeNulls: boolean\n :return: (io.deephaven.api.agg.spec.AggSpecDistinct) AggSpecDistinct.of(boolean) for includeNulls", + "first": ":return: (io.deephaven.api.agg.spec.AggSpecFirst) AggSpecFirst.of()", + "formula": "*Overload 1* \n :param formula: java.lang.String\n :return: (io.deephaven.api.agg.spec.AggSpecFormula) AggSpecFormula.of(String) for formula\n \n*Overload 2* \n :param formula: java.lang.String\n :param paramToken: java.lang.String\n :return: (io.deephaven.api.agg.spec.AggSpecFormula) AggSpecFormula.of(String, String) for formula and paramToken", + "freeze": ":return: (io.deephaven.api.agg.spec.AggSpecFreeze) AggSpecFreeze.of()", + "group": ":return: (io.deephaven.api.agg.spec.AggSpecGroup) AggSpecGroup.of()", + "last": ":return: (io.deephaven.api.agg.spec.AggSpecLast) AggSpecLast.of()", + "max": ":return: (io.deephaven.api.agg.spec.AggSpecMax) AggSpecMax.of()", + "median": "*Overload 1* \n :return: (io.deephaven.api.agg.spec.AggSpecMedian) AggSpecMedian.of()\n \n*Overload 2* \n :param averageEvenlyDivided: boolean\n :return: (io.deephaven.api.agg.spec.AggSpecMedian) AggSpecMedian.of(boolean) for averageEvenlyDivided", + "min": ":return: (io.deephaven.api.agg.spec.AggSpecMin) AggSpecMin.of()", + "percentile": "*Overload 1* \n :param percentile: double\n :return: (io.deephaven.api.agg.spec.AggSpecPercentile) AggSpecPercentile.of(double) for percentile\n \n*Overload 2* \n :param percentile: double\n :param averageEvenlyDivided: boolean\n :return: (io.deephaven.api.agg.spec.AggSpecPercentile) AggSpecPercentile.of(double, boolean) for percentile and averageEvenlyDivided", + "sortedFirst": "*Overload 1* \n :param columns: java.lang.String...\n :return: (io.deephaven.api.agg.spec.AggSpecSortedFirst) AggSpecSortedFirst for the supplied sort columns\n \n*Overload 2* \n :param columns: java.util.Collection\n :return: (io.deephaven.api.agg.spec.AggSpecSortedFirst) AggSpecSortedFirst for the supplied sort columns", + "sortedLast": "*Overload 1* \n :param columns: java.lang.String...\n :return: (io.deephaven.api.agg.spec.AggSpecSortedLast) AggSpecSortedLast for the supplied sort columns\n \n*Overload 2* \n :param columns: java.util.Collection\n :return: (io.deephaven.api.agg.spec.AggSpecSortedLast) AggSpecSortedLast for the supplied sort columns", + "std": ":return: (io.deephaven.api.agg.spec.AggSpecStd) AggSpecStd.of()", + "sum": ":return: (io.deephaven.api.agg.spec.AggSpecSum) AggSpecSum.of()", + "tDigest": "*Overload 1* \n :return: (io.deephaven.api.agg.spec.AggSpecTDigest) AggSpecTDigest.of()\n \n*Overload 2* \n :param compression: double\n :return: (io.deephaven.api.agg.spec.AggSpecTDigest) AggSpecTDigest.of(double) for compression", + "unique": "*Overload 1* \n :return: (io.deephaven.api.agg.spec.AggSpecUnique) AggSpecUnique.of()\n \n*Overload 2* \n :param includeNulls: boolean\n :param nonUniqueSentinel: java.lang.Object\n :return: (io.deephaven.api.agg.spec.AggSpecUnique) AggSpecUnique.of(boolean, Object) for includeNulls and nonUniqueSentinel", + "var": ":return: (io.deephaven.api.agg.spec.AggSpecVar) AggSpecVar.of()", + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor", + "wavg": ":param weightColumn: java.lang.String\n:return: (io.deephaven.api.agg.spec.AggSpecWAvg) AggSpecWAvg.of(ColumnName) for the supplied weightColumn", + "wsum": ":param weightColumn: java.lang.String\n:return: (io.deephaven.api.agg.spec.AggSpecWSum) AggSpecWSum.of(ColumnName) for the supplied weightColumn" }, "path": "io.deephaven.api.agg.spec.AggSpec", - "text": "An aggregation specification represents the configurable options for column aggregations.", + "text": "An aggregation specification represents the configurable options for singular and\n compound column aggregations.", "typeName": "interface" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json index 488ab2fca2e..da06d7d7481 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpec/Visitor.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpec$Visitor", "methods": { - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param freeze: io.deephaven.api.agg.spec.AggSpecFreeze\n \n*Overload 6* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 7* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 8* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 9* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 10* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 11* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 12* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 13* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 14* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 15* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 16* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 17* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 18* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 19* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 20* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 21* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 22* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 23* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" + "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 4* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 5* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 6* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 7* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 8* \n :param freeze: io.deephaven.api.agg.spec.AggSpecFreeze\n \n*Overload 9* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 10* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 11* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 12* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 13* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 14* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 15* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 16* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 17* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 18* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 19* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 20* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 21* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 22* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 23* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" }, "path": "io.deephaven.api.agg.spec.AggSpec.Visitor", "typeName": "interface" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json index 75457dbd478..09891bcf4d0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAbsSum.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecAbsSum", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecAbsSum", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecAbsSum", + "text": "Specifies an aggregation that outputs the sum of absolute input values for each group. Only works with numeric input\n types and Boolean.\n \nBoolean inputs are aggregated according to the following rules:\n \n* If any input value is true, the output value is true\n* If there are no non-null input values, the output value is null\n* Else all input values must be false, and the output value is false", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json index c7af90b967f..e4a6d2de633 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecApproximatePercentile.json @@ -2,12 +2,12 @@ "className": "io.deephaven.api.agg.spec.AggSpecApproximatePercentile", "methods": { "compression": "T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large.\n\n:return: (double) The T-Digest compression factor", - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 2* \n :param percentile: double\n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecApproximatePercentile", "percentile": "Percentile. Must be in range [0.0, 1.0].\n\n:return: (double) The percentile", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecApproximatePercentile", - "text": "Approximate percentile aggregation using a T-Digest for calculation. Efficiently supports multiple output percentiles\n based on a single input column. May only be used on static or add-only tables.", + "text": "Specifies an aggregation that outputs a percentile approximated using a T-Digest with the specified\n compression().\n \n Efficiently supports multiple output percentiles based on a single input column.\n \n May only be used on static or add-only tables.\n \n Only supported for numeric types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json index 07adebd765f..68decbdfcd6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecAvg.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecAvg", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecAvg", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecAvg", + "text": "Specifies an aggregation that outputs the arithmetic mean for each group. Only works with numeric input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json index ed7fd14cfc5..8eebf5155f0 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecBase.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.api.agg.spec.AggSpecBase", "methods": { - "aggregation": "*Overload 1* \n :param pair: io.deephaven.api.agg.Pair\n :return: io.deephaven.api.agg.ColumnAggregation\n \n*Overload 2* \n :param pairs: io.deephaven.api.agg.Pair...\n :return: io.deephaven.api.agg.Aggregation\n \n*Overload 3* \n :param pairs: java.util.Collection\n :return: io.deephaven.api.agg.Aggregation" + "aggregation": "**Incompatible overloads text - text from the first overload:**\n\nBuild a ColumnAggregation for this AggSpec.\n\n*Overload 1* \n :param pair: (io.deephaven.api.agg.Pair) - The input/output column name pair\n :return: (io.deephaven.api.agg.ColumnAggregation) The aggregation\n \n*Overload 2* \n :param pairs: (io.deephaven.api.agg.Pair...) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation\n \n*Overload 3* \n :param pairs: (java.util.Collection) - The input/output column name pairs\n :return: (io.deephaven.api.agg.Aggregation) The aggregation" }, "path": "io.deephaven.api.agg.spec.AggSpecBase", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json index f65fcd86e09..5b705162fc9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecColumnReferences.json @@ -3,7 +3,7 @@ "methods": { "of": ":param spec: io.deephaven.api.agg.spec.AggSpec\n:return: java.util.Set", "out": ":return: java.util.Set", - "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 4* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 5* \n :param freeze: io.deephaven.api.agg.spec.AggSpecFreeze\n \n*Overload 6* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 7* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 8* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 9* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 10* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 11* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 12* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 13* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 14* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 15* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 16* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 17* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 18* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 19* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 20* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 21* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 22* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 23* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" + "visit": "*Overload 1* \n :param absSum: io.deephaven.api.agg.spec.AggSpecAbsSum\n \n*Overload 2* \n :param approxPct: io.deephaven.api.agg.spec.AggSpecApproximatePercentile\n \n*Overload 3* \n :param avg: io.deephaven.api.agg.spec.AggSpecAvg\n \n*Overload 4* \n :param countDistinct: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 5* \n :param distinct: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 6* \n :param first: io.deephaven.api.agg.spec.AggSpecFirst\n \n*Overload 7* \n :param formula: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 8* \n :param freeze: io.deephaven.api.agg.spec.AggSpecFreeze\n \n*Overload 9* \n :param group: io.deephaven.api.agg.spec.AggSpecGroup\n \n*Overload 10* \n :param last: io.deephaven.api.agg.spec.AggSpecLast\n \n*Overload 11* \n :param max: io.deephaven.api.agg.spec.AggSpecMax\n \n*Overload 12* \n :param median: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 13* \n :param min: io.deephaven.api.agg.spec.AggSpecMin\n \n*Overload 14* \n :param pct: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 15* \n :param sortedFirst: io.deephaven.api.agg.spec.AggSpecSortedFirst\n \n*Overload 16* \n :param sortedLast: io.deephaven.api.agg.spec.AggSpecSortedLast\n \n*Overload 17* \n :param std: io.deephaven.api.agg.spec.AggSpecStd\n \n*Overload 18* \n :param sum: io.deephaven.api.agg.spec.AggSpecSum\n \n*Overload 19* \n :param tDigest: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 20* \n :param unique: io.deephaven.api.agg.spec.AggSpecUnique\n \n*Overload 21* \n :param wAvg: io.deephaven.api.agg.spec.AggSpecWAvg\n \n*Overload 22* \n :param wSum: io.deephaven.api.agg.spec.AggSpecWSum\n \n*Overload 23* \n :param var: io.deephaven.api.agg.spec.AggSpecVar" }, "path": "io.deephaven.api.agg.spec.AggSpecColumnReferences", "typeName": "class" diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json index 7399690a121..4c7b7ced325 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecCountDistinct.json @@ -1,11 +1,12 @@ { "className": "io.deephaven.api.agg.spec.AggSpecCountDistinct", "methods": { - "countNulls": ":return: boolean", - "description": ":return: java.lang.String", + "countNulls": "Whether null input values should be included when counting the distinct input values.\n\n:return: (boolean) Whether to count nulls", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct\n \n*Overload 2* \n :param countNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecCountDistinct", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecCountDistinct", + "text": "Specifies an aggregation that outputs the count of distinct values for each group.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json index a78f5b47bd9..d24e5dfe42e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecDistinct.json @@ -1,11 +1,12 @@ { "className": "io.deephaven.api.agg.spec.AggSpecDistinct", "methods": { - "description": ":return: java.lang.String", - "includeNulls": ":return: boolean", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "includeNulls": "Whether null input values should be included in the distinct output values.\n\n:return: (boolean) Whether to include nulls", "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecDistinct\n \n*Overload 2* \n :param includeNulls: boolean\n :return: io.deephaven.api.agg.spec.AggSpecDistinct", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecDistinct", + "text": "Specifies an aggregation that outputs the distinct values for each group as a Deephaven vector\n (io.deephaven.vector.Vector).", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json index 973aa755a0a..d3e540418b9 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFirst.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecFirst", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecFirst", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecFirst", + "text": "Specifies an aggregation that outputs the first value in the input column for each group.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json index 557840f37f3..8b5c854d759 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFormula.json @@ -1,12 +1,13 @@ { "className": "io.deephaven.api.agg.spec.AggSpecFormula", "methods": { - "description": ":return: java.lang.String", - "formula": ":return: java.lang.String", - "formulaParam": ":return: java.lang.String", - "of": "*Overload 1* \n :param formula: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 2* \n :param formula: java.lang.String\n :param formulaParam: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "formula": "The formula to use to calculate output values from grouped input values.\n\n:return: (java.lang.String) The formula", + "of": "*Overload 1* \n :param formula: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula\n \n*Overload 2* \n :param formula: java.lang.String\n :param paramToken: java.lang.String\n :return: io.deephaven.api.agg.spec.AggSpecFormula", + "paramToken": "The formula parameter token to be replaced with the input column name for evaluation.\n\n:return: (java.lang.String) The parameter token", + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecFormula", + "text": "Specifies an aggregation that applies a formula to each input group (as a Deephaven vector\n (io.deephaven.vector.Vector)) to produce the corresponding output value. Each input column name is substituted for\n the param token for evaluation.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json index 1605da95d4d..e9b6b62a114 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecFreeze.json @@ -1,11 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecFreeze", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecFreeze", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecFreeze", - "text": "Aggregation that freezes the first value for each group and ignores subsequent changes. When groups are removed, the\n corresponding output row is removed. When groups are re-added (on a subsequent update cycle), the newly added value\n is then frozen.\n \n Only one row per group is allowed in the output, because the operation has no way to determine which row to freeze\n otherwise. This is a constraint on the input data.", + "text": "Specifies an aggregation that freezes the first value for each group and ignores subsequent changes. When groups are\n removed, the corresponding output row is removed. When groups are re-added (on a subsequent update cycle), the newly\n added value is then frozen.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json index 539e9131483..266d3f9e62a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecGroup.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecGroup", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecGroup", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecGroup", + "text": "Specifies an aggregation that outputs each group of input values as a Deephaven vector (io.deephaven.vector.Vector).", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json index 7fd81792f01..3bf0fac91de 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecLast.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecLast", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecLast", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecLast", + "text": "Specifies an aggregation that outputs the last value in the input column for each group.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json index c08f4db1bbf..10d362ecdc3 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMax.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecMax", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecMax", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecMax", + "text": "Specifies an aggregation that outputs the maximum value in the input column for each group. Only works for numeric or\n Comparable input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json index 99a88476eea..9f400f65523 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMedian.json @@ -1,11 +1,12 @@ { "className": "io.deephaven.api.agg.spec.AggSpecMedian", "methods": { - "averageMedian": ":return: boolean", - "description": ":return: java.lang.String", - "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 2* \n :param averageMedian: boolean\n :return: io.deephaven.api.agg.spec.AggSpecMedian", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "averageEvenlyDivided": "Whether to average the highest low-bucket value and lowest high-bucket value, when the low-bucket and high-bucket\n are of equal size. Only applies to numeric types.\n\n:return: (boolean) Whether to average the two result candidates for evenly-divided input sets of numeric types", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecMedian\n \n*Overload 2* \n :param averageEvenlyDivided: boolean\n :return: io.deephaven.api.agg.spec.AggSpecMedian", + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecMedian", + "text": "Specifier for a column aggregation that produces a median value from the input column's values for each group. Only\n works for numeric or Comparable input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json index d76949f0c02..854c393bc0f 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecMin.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecMin", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecMin", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecMin", + "text": "Specifies an aggregation that outputs the minimum value in the input column for each group. Only works for numeric or\n Comparable input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json index 45eebefd3bc..ae8a5c0975e 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecPercentile.json @@ -1,12 +1,13 @@ { "className": "io.deephaven.api.agg.spec.AggSpecPercentile", "methods": { - "averageMedian": ":return: boolean", - "description": ":return: java.lang.String", - "of": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 2* \n :param percentile: double\n :param averageMedian: boolean\n :return: io.deephaven.api.agg.spec.AggSpecPercentile", - "percentile": ":return: double", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "averageEvenlyDivided": "Whether to average the highest low-bucket value and lowest high-bucket value, when the low-bucket and high-bucket\n are of equal size. Only applies to numeric types.\n\n:return: (boolean) Whether to average the two result candidates for evenly-divided input sets of numeric types", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "of": "*Overload 1* \n :param percentile: double\n :return: io.deephaven.api.agg.spec.AggSpecPercentile\n \n*Overload 2* \n :param percentile: double\n :param averageEvenlyDivided: boolean\n :return: io.deephaven.api.agg.spec.AggSpecPercentile", + "percentile": "The percentile to calculate. Must be >= 0.0 and <= 1.0.\n\n:return: (double) The percentile to calculate", + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecPercentile", + "text": "Specifier for a column aggregation that produces a percentile value from the input column's values for each group.\n Only works for numeric or Comparable input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json index fb53104186f..c1bc2327c74 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedFirst.json @@ -2,11 +2,11 @@ "className": "io.deephaven.api.agg.spec.AggSpecSortedFirst", "methods": { "builder": ":return: io.deephaven.api.agg.spec.AggSpecSortedFirst.Builder", - "columns": ":return: java.util.List", - "description": ":return: java.lang.String", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "columns": "The columns to sort on to determine the order within each group.\n\n:return: (java.util.List) The sort columns", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecSortedFirst", - "text": "Note: the sorted-first aggregation only supports SortColumn.Order.ASCENDING columns at the moment.", + "text": "Specifies an aggregation that outputs the first value in the input column for each group, after sorting the group on\n the sort columns.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json index 93aa506d481..ca4619ce3da 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSortedLast.json @@ -2,11 +2,11 @@ "className": "io.deephaven.api.agg.spec.AggSpecSortedLast", "methods": { "builder": ":return: io.deephaven.api.agg.spec.AggSpecSortedLast.Builder", - "columns": ":return: java.util.List", - "description": ":return: java.lang.String", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "columns": "The columns to sort on to determine the order within each group.\n\n:return: (java.util.List) The sort columns", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecSortedLast", - "text": "Note: the sorted-last aggregation only supports SortColumn.Order.ASCENDING columns at the moment.", + "text": "Specifies an aggregation that outputs the last value in the input column for each group, after sorting the group on\n the sort columns.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json index 4a080a83020..8157735d2bf 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecStd.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecStd", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecStd", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecStd", + "text": "Specifies an aggregation that outputs the standard deviation of the input column values for each group. Only works\n for numeric input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json index aa2090c5c6c..4216107c2cb 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecSum.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecSum", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecSum", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecSum", + "text": "Specifies an aggregation that outputs the sum of input values for each group. Only works with numeric input types and\n Boolean.\n \nBoolean inputs are aggregated according to the following rules:\n \n* If any input value is true, the output value is true\n* If there are no non-null input values, the output value is null\n* Else all input values must be false, and the output value is false", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json index 15ad2c204bf..08a7fc15395 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecTDigest.json @@ -2,11 +2,11 @@ "className": "io.deephaven.api.agg.spec.AggSpecTDigest", "methods": { "compression": "T-Digest compression factor. Must be greater than or equal to 1. Defaults to 100. 1000 is extremely large.\n\n:return: (double) The T-Digest compression factor", - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": "*Overload 1* \n :return: io.deephaven.api.agg.spec.AggSpecTDigest\n \n*Overload 2* \n :param compression: double\n :return: io.deephaven.api.agg.spec.AggSpecTDigest", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecTDigest", - "text": "Aggregates input column into a TDigest output column with the specified compression.\n \n May be used to implement parallel percentile calculations by splitting inputs and accumulating results into a single\n downstream TDigest.\n \n May only be used on static or add-only tables.", + "text": "Specifies an aggregation that outputs a T-Digest (com.tdunning.math.stats.TDigest) with the specified\n compression().\n \n May be used to implement parallel percentile calculations by splitting inputs and accumulating results into a single\n downstream TDigest.\n \n May only be used on static or add-only tables.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json index 89067f0e761..38e1297025a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecUnique.json @@ -1,12 +1,13 @@ { "className": "io.deephaven.api.agg.spec.AggSpecUnique", "methods": { - "description": ":return: java.lang.String", - "includeNulls": ":return: boolean", - "nonUniqueSentinel": ":return: java.lang.Object", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", + "includeNulls": "Whether to include null values as a distinct value for determining if there is only one unique value to\n output.\n\n:return: (boolean) Whether to include nulls", + "nonUniqueSentinel": "The output value to use for groups that don't have a single unique input value.\n\n:return: (java.lang.Object) The non-unique sentinel value", "of": "**Incompatible overloads text - text from the first overload:**\n\nSpecify a \"unique\" aggregation that does not treat null as a value for purposes of determining if the\n values in a group are unique. If a group is non-empty but contains only null values, its result will be\n null. If a group contains more than a single unique value, its result will also be null.\n\n*Overload 1* \n :return: (io.deephaven.api.agg.spec.AggSpecUnique) The \"unique\" aggregation specification\n \n*Overload 2* \n :param includeNulls: (boolean) - Whether null is treated as a value for determining if the values in a group are\n unique\n :param nonUniqueSentinel: (java.lang.Object) - Sentinel value to use if a group contains more than a single unique value\n :return: (io.deephaven.api.agg.spec.AggSpecUnique) The \"unique\" aggregation specification", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecUnique", + "text": "Specifies an aggregation that outputs the single unique input value for groups that have one, null if all\n input values are null, or nonUniqueSentinel() if there is more than one distinct value.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json index 4f71929e811..caf5b893a20 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecVar.json @@ -1,10 +1,11 @@ { "className": "io.deephaven.api.agg.spec.AggSpecVar", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":return: io.deephaven.api.agg.spec.AggSpecVar", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.api.agg.spec.AggSpecVar", + "text": "Specifies an aggregation that outputs the variance of the input column values for each group. Only works for numeric\n input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json index 8722797fba1..55628840b46 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWAvg.json @@ -1,11 +1,12 @@ { "className": "io.deephaven.api.agg.spec.AggSpecWAvg", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":param weight: io.deephaven.api.ColumnName\n:return: io.deephaven.api.agg.spec.AggSpecWAvg", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V", - "weight": ":return: io.deephaven.api.ColumnName" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor", + "weight": "Column name for the source of input weights.\n\n:return: (io.deephaven.api.ColumnName) The weight column name" }, "path": "io.deephaven.api.agg.spec.AggSpecWAvg", + "text": "Specifies an aggregation that outputs the average (arithmetic mean) of the input column values weighted by the\n weight column values for each group. Only works for numeric input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json index 0e8411a5684..0b05c9660e6 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json +++ b/Integrations/python/deephaven/doc/io/deephaven/api/agg/spec/AggSpecWSum.json @@ -1,11 +1,12 @@ { "className": "io.deephaven.api.agg.spec.AggSpecWSum", "methods": { - "description": ":return: java.lang.String", + "description": "Get a simple description for this AggSpec.\n\n:return: (java.lang.String) The description", "of": ":param weight: io.deephaven.api.ColumnName\n:return: io.deephaven.api.agg.spec.AggSpecWSum", - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V", - "weight": ":return: io.deephaven.api.ColumnName" + "walk": "Glue method to deliver this AggSpec to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor", + "weight": "Column name for the source of input weights.\n\n:return: (io.deephaven.api.ColumnName) The weight column name" }, "path": "io.deephaven.api.agg.spec.AggSpecWSum", + "text": "Specifies an aggregation that outputs the sum of the input column values weighted by the weight\n column values for each group. Only works for numeric input types.", "typeName": "class" } \ No newline at end of file diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json index f1dc6201438..c67225d959a 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregation.json @@ -3,7 +3,7 @@ "methods": { "nullColumns": ":param resultColumns: java.util.Map>\n:return: io.deephaven.engine.table.impl.by.rollup.RollupAggregation", "partition": ":param includeConstituents: boolean\n:return: io.deephaven.engine.table.impl.by.rollup.RollupAggregation", - "walk": "*Overload 1* \n Note: Java generics information - \n \n :param visitor: V\n :return: V\n \n*Overload 2* \n Note: Java generics information - \n \n :param visitor: V\n :return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\n*Overload 1* \n Note: Java generics information - \n \n :param visitor: (V) - The visitor\n :return: (V) The visitor\n \n*Overload 2* \n Note: Java generics information - \n \n :param visitor: V\n :return: V" }, "path": "io.deephaven.engine.table.impl.by.rollup.RollupAggregation", "text": "Rollup-specific aggregations.", diff --git a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json index 14e57a783c1..5dd273de1f4 100644 --- a/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json +++ b/Integrations/python/deephaven/doc/io/deephaven/engine/table/impl/by/rollup/RollupAggregationBase.json @@ -1,7 +1,7 @@ { "className": "io.deephaven.engine.table.impl.by.rollup.RollupAggregationBase", "methods": { - "walk": "Note: Java generics information - \n\n:param visitor: V\n:return: V" + "walk": "Glue method to deliver this Aggregation to a AggSpec.Visitor.\n\nNote: Java generics information - \n\n:param visitor: (V) - The visitor\n:return: (V) The visitor" }, "path": "io.deephaven.engine.table.impl.by.rollup.RollupAggregationBase", "typeName": "class" diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java index 4d36f504f76..bd80ce26a99 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -726,7 +726,8 @@ public void visit(@NotNull final AggSpecMin min) { @Override public void visit(@NotNull final AggSpecPercentile pct) { - addBasicOperators((t, n) -> new SsmChunkedPercentileOperator(t, pct.percentile(), pct.averageEvenlyDivided(), n)); + addBasicOperators( + (t, n) -> new SsmChunkedPercentileOperator(t, pct.percentile(), pct.averageEvenlyDivided(), n)); } @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index b468a2d98c6..5f54e9204e3 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java +++ b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java @@ -269,8 +269,8 @@ static Aggregation AggFirstRowKey(String resultColumn) { * * @param formula The {@link AggSpecFormula#formula() formula} to use for all input columns to produce all output * columns - * @param paramToken The {@link AggSpecFormula#paramToken() parameter token} to replace with the input column - * name in {@code formula} + * @param paramToken The {@link AggSpecFormula#paramToken() parameter token} to replace with the input column name + * in {@code formula} * @param pairs The input/output column name pairs * @return The aggregation */ diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 991c2d2a781..f15b2ecaf98 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -293,8 +293,8 @@ static AggSpecWSum wsum(String weightColumn) { V walk(V visitor); /* - * Visitor interface. Combines with {@link #walk(Visitor) in order to allow for type-safe AggSpec evaluation - * without switch statements or if-else blocks. + * Visitor interface. Combines with {@link #walk(Visitor) in order to allow for type-safe AggSpec evaluation without + * switch statements or if-else blocks. */ interface Visitor { void visit(AggSpecAbsSum absSum);