From 83e0c972877c52aa660770f02e8454a9ed4525c3 Mon Sep 17 00:00:00 2001 From: Larry Booker Date: Wed, 18 Dec 2024 16:44:35 -0800 Subject: [PATCH] feat: implement `AggCountWhere` to support generic counting using `Filter` (#6497) ## Groovy Examples ``` Table countTable = table.aggBy(List.of( AggCountWhere("filter1", "intCol >= 50"), AggCountWhere("filter2", "intCol >= 50", "intCol != 80"), AggCountWhere("filter3", Filter.or(Filter.from("intCol >= 50", "intCol == 3"))), AggCountWhere("filter4", "true"), AggCountWhere("filter5", "false"), AggCountWhere("filter6", "intCol % 2 == 0"), AggCountWhere("filter7", Filter.and(Filter.or(Filter.from("false", "intCol % 3 == 0")), Filter.or(Filter.from("false", "intCol % 2 == 0")))), AggCountWhere("filter8", "intCol % 2 == 0", "intCol % 3 == 0"), AggCountWhere("filter9", Filter.and(Filter.and(Filter.from("intCol > 0")), Filter.and(Filter.from("intCol <= 10", "intCol >= 5")))), // Multiple input columns AggCountWhere("filter10", "intCol >= 5", "doubleCol <= 10.0"), AggCountWhere("filter11", "intCol >= 5 && intColNulls != 3 && doubleCol <= 10.0"), // DynamicWhereFilter AggCountWhere("filter12", new DynamicWhereFilter(setTable, true, MatchPairFactory.getExpressions("intCol")))), "Sym").sort("Sym"); ``` ## Python Examples ``` from deephaven import empty_table from deephaven.agg import count_where table_size = 120000000 table = empty_table(table_size).update(["int1M=randomInt(0,1000000)", "int640=randomInt(0,640)", "int250=randomInt(0,250)"]).select() # zero-key t_count = table.agg_by(aggs=count_where(col="count", filters=["int1M < 500000", "int1M > 499000"])) # bucketed t_count = table.agg_by(aggs=count_where(col="count", filters=["int1M < 500000", "int1M > 499000"]), by="int250") ``` --- .../table/impl/by/AggregationProcessor.java | 71 +- .../table/impl/by/CountWhereOperator.java | 574 ++++ .../impl/by/RecordingInternalOperator.java | 162 + .../impl/chunkfilter/ByteChunkFilter.java | 67 + .../ByteChunkMatchFilterFactory.java | 111 +- .../impl/chunkfilter/ByteRangeComparator.java | 66 +- .../impl/chunkfilter/CharChunkFilter.java | 63 + .../CharChunkMatchFilterFactory.java | 111 +- .../impl/chunkfilter/CharRangeComparator.java | 66 +- .../table/impl/chunkfilter/ChunkFilter.java | 144 +- .../impl/chunkfilter/DoubleChunkFilter.java | 67 + .../DoubleChunkMatchFilterFactory.java | 111 +- .../chunkfilter/DoubleRangeComparator.java | 66 +- .../impl/chunkfilter/FloatChunkFilter.java | 67 + .../FloatChunkMatchFilterFactory.java | 111 +- .../chunkfilter/FloatRangeComparator.java | 66 +- .../impl/chunkfilter/IntChunkFilter.java | 67 + .../IntChunkMatchFilterFactory.java | 111 +- .../impl/chunkfilter/IntRangeComparator.java | 66 +- .../impl/chunkfilter/LongChunkFilter.java | 67 + .../LongChunkMatchFilterFactory.java | 111 +- .../impl/chunkfilter/LongRangeComparator.java | 70 +- .../impl/chunkfilter/ObjectChunkFilter.java | 63 + .../ObjectChunkMatchFilterFactory.java | 110 +- .../impl/chunkfilter/ShortChunkFilter.java | 67 + .../ShortChunkMatchFilterFactory.java | 111 +- .../chunkfilter/ShortRangeComparator.java | 66 +- .../StringChunkMatchFilterFactory.java | 121 +- .../SupportedRangeJoinAggregations.java | 5 + .../rangejoin/ValidFloatingPointFilter.java | 40 +- .../impl/select/AbstractConditionFilter.java | 69 +- .../impl/select/AbstractRangeFilter.java | 7 +- .../impl/select/ComparableRangeFilter.java | 103 +- .../table/impl/select/ConditionFilter.java | 171 +- .../table/impl/select/ExposesChunkFilter.java | 17 + .../table/impl/select/InstantRangeFilter.java | 44 +- .../engine/table/impl/select/MatchFilter.java | 16 +- .../engine/table/impl/select/RangeFilter.java | 13 +- .../SingleSidedComparableRangeFilter.java | 64 +- .../impl/select/WhereFilterPatternImpl.java | 123 +- .../FilterKernelPythonChunkedFunction.java | 55 + .../FilterKernelPythonSingularFunction.java | 45 + .../impl/QueryLibraryImportsDefaults.java | 2 + .../table/impl/QueryTableWhereTest.java | 46 +- .../engine/table/impl/TestAggBy.java | 229 +- .../impl/select/FilterKernelArraySample.java | 36 +- .../table/impl/select/FilterKernelSample.java | 44 +- .../impl/select/FormulaKernelSample.java | 2 + .../table/impl/select/FormulaSample.java | 2 + go/internal/proto/table/table.pb.go | 2735 +++++++++-------- .../io/deephaven/client/AggBySessionTest.java | 15 +- .../client/impl/AggregationBuilder.java | 21 +- .../proto/deephaven_core/proto/table.proto | 8 + py/client/deephaven_core/proto/table_pb2.py | 190 +- py/client/pydeephaven/agg.py | 25 + py/client/tests/test_table.py | 26 +- py/server/deephaven/agg.py | 19 +- py/server/tests/test_table.py | 26 +- .../replicators/ReplicateChunkFilters.java | 5 + .../server/table/ops/AggregationAdapter.java | 24 +- .../io/deephaven/api/agg/Aggregation.java | 35 +- .../api/agg/AggregationDescriptions.java | 5 + .../api/agg/AggregationOptimizer.java | 6 + .../deephaven/api/agg/AggregationOutputs.java | 5 + .../deephaven/api/agg/AggregationPairs.java | 5 + .../java/io/deephaven/api/agg/CountWhere.java | 45 + .../io/deephaven/api/agg/AggregationTest.java | 5 + 67 files changed, 4340 insertions(+), 2946 deletions(-) create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountWhereOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/by/RecordingInternalOperator.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkFilter.java create mode 100644 engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java create mode 100644 table-api/src/main/java/io/deephaven/api/agg/CountWhere.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 3bf51eb5e34..2d4e0908cd7 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 @@ -92,6 +92,7 @@ import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; import io.deephaven.engine.table.impl.by.ssmpercentile.SsmChunkedPercentileOperator; import io.deephaven.engine.table.impl.select.SelectColumn; +import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.sources.ReinterpretUtils; import io.deephaven.engine.table.impl.ssms.SegmentedSortedMultiSet; import io.deephaven.engine.table.impl.util.freezeby.FreezeByCountOperator; @@ -107,13 +108,7 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.time.Instant; -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.Set; +import java.util.*; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -693,6 +688,56 @@ public void visit(@NotNull final Count count) { addNoInputOperator(new CountAggregationOperator(count.column().name())); } + @Override + public void visit(@NotNull final CountWhere countWhere) { + final WhereFilter[] whereFilters = WhereFilter.fromInternal(countWhere.filter()); + + final Map inputColumnRecorderMap = new HashMap<>(); + final List recorderList = new ArrayList<>(); + final List filterRecorderList = new ArrayList<>(); + + // Verify all the columns in the where filters are present in the table and valid for use. + for (final WhereFilter whereFilter : whereFilters) { + whereFilter.init(table.getDefinition()); + if (whereFilter.isRefreshing()) { + throw new UnsupportedOperationException("AggCountWhere does not support refreshing filters"); + } + + // Compute which recording operators this filter will use. + final List inputColumnNames = whereFilter.getColumns(); + final int inputColumnCount = whereFilter.getColumns().size(); + final RecordingInternalOperator[] recorders = new RecordingInternalOperator[inputColumnCount]; + for (int ii = 0; ii < inputColumnCount; ++ii) { + final String inputColumnName = inputColumnNames.get(ii); + final RecordingInternalOperator recorder = + inputColumnRecorderMap.computeIfAbsent(inputColumnName, k -> { + // Create a recording operator for the column and add it to the list of operators. + final ColumnSource inputSource = table.getColumnSource(inputColumnName); + final RecordingInternalOperator newRecorder = + new RecordingInternalOperator(inputColumnName, inputSource); + recorderList.add(newRecorder); + return newRecorder; + }); + recorders[ii] = recorder; + } + filterRecorderList.add(recorders); + } + + final RecordingInternalOperator[] recorders = recorderList.toArray(RecordingInternalOperator[]::new); + final RecordingInternalOperator[][] filterRecorders = + filterRecorderList.toArray(RecordingInternalOperator[][]::new); + final String[] inputColumnNames = + inputColumnRecorderMap.keySet().toArray(ArrayTypeUtils.EMPTY_STRING_ARRAY); + + // Add the recording operators, making them dependent on all input columns so they all are populated if any + // are modified + for (final RecordingInternalOperator recorder : recorders) { + addOperator(recorder, recorder.getInputColumnSource(), inputColumnNames); + } + addOperator(new CountWhereOperator(countWhere.column().name(), whereFilters, recorders, filterRecorders), + null, inputColumnNames); + } + @Override public void visit(@NotNull final FirstRowKey firstRowKey) { addFirstOrLastOperators(true, firstRowKey.column().name()); @@ -1004,6 +1049,11 @@ public void visit(@NotNull final Count count) { addNoInputOperator(new CountAggregationOperator(count.column().name())); } + @Override + public void visit(@NotNull final CountWhere countWhere) { + addNoInputOperator(new CountAggregationOperator(countWhere.column().name())); + } + @Override public void visit(@NotNull final NullColumns nullColumns) { transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns())); @@ -1149,6 +1199,13 @@ public void visit(@NotNull final Count count) { addOperator(makeSumOperator(resultSource.getType(), resultName, false), resultSource, resultName); } + @Override + public void visit(@NotNull final CountWhere countWhere) { + final String resultName = countWhere.column().name(); + final ColumnSource resultSource = table.getColumnSource(resultName); + addOperator(makeSumOperator(resultSource.getType(), resultName, false), resultSource, resultName); + } + @Override public void visit(@NotNull final NullColumns nullColumns) { transformers.add(new NullColumnAggregationTransformer(nullColumns.resultColumns())); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountWhereOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountWhereOperator.java new file mode 100644 index 00000000000..07319b61642 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/CountWhereOperator.java @@ -0,0 +1,574 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.by; + +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.chunk.attributes.Values; +import io.deephaven.engine.rowset.*; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.select.*; +import io.deephaven.engine.table.impl.sources.LongArraySource; +import io.deephaven.engine.table.impl.sources.chunkcolumnsource.ChunkColumnSource; +import io.deephaven.util.SafeCloseable; +import io.deephaven.util.SafeCloseableArray; +import org.jetbrains.annotations.NotNull; + +import java.util.*; + +import static io.deephaven.engine.util.NullSafeAddition.plusLong; +import static io.deephaven.engine.util.TableTools.emptyTable; + +/** + * Implements a counting operator that counts the number of rows that pass a set of filters. Chunk data is accessed + * through {@link RecordingInternalOperator recorder} instances. + */ + +public class CountWhereOperator implements IterativeChunkedAggregationOperator { + /** + * The output column name in the result table. + */ + private final String resultName; + /** + * The output column source in the result table. + */ + private final LongArraySource resultColumnSource; + /** + * The filters to apply to the data. + */ + private final CountFilter[] filters; + /** + * The recorder instances that provide the chunk data. + */ + private final RecordingInternalOperator[] recorders; + /** + * A table constructed from chunk sources, populated with incoming chunk data and used to evaluate filters. + */ + private final Table chunkSourceTable; + /** + * The chunk sources that populate the chunkSourceTable. + */ + private final ChunkColumnSource[] chunkColumnSources; + /** + * Track whether the chunkSourceTable needs to be updated with new data. + */ + private final boolean updateChunkSourceTable; + + /** + * Internal representation of the filter to apply to the data. + */ + private static class CountFilter { + private final ChunkFilter chunkFilter; + private final AbstractConditionFilter.Filter conditionFilter; + private final WhereFilter whereFilter; + private final RecordingInternalOperator[] recorders; + + public CountFilter(ChunkFilter chunkFilter, RecordingInternalOperator[] recorders) { + this.chunkFilter = chunkFilter; + this.conditionFilter = null; + this.whereFilter = null; + this.recorders = recorders; + } + + public CountFilter(AbstractConditionFilter.Filter conditionFilter, RecordingInternalOperator[] recorders) { + this.chunkFilter = null; + this.conditionFilter = conditionFilter; + this.whereFilter = null; + this.recorders = recorders; + } + + public CountFilter(WhereFilter whereFilter, RecordingInternalOperator[] recorders) { + this.chunkFilter = null; + this.conditionFilter = null; + this.whereFilter = whereFilter; + this.recorders = recorders; + } + } + + static class BaseContext implements SafeCloseable { + /** + * ConditionFilter needs a context to store intermediate results. + */ + final ConditionFilter.FilterKernel.Context[] conditionFilterContexts; + /** + * Contains the results of the filters as a boolean chunk, where true indicates a row passed all filters. + */ + final WritableBooleanChunk resultsChunk; + /** + * The chunk data from the recorders to be used as input to the filters. + */ + final Chunk[][] filterChunks; + + BaseContext(final int size, final CountFilter[] filters) { + conditionFilterContexts = new ConditionFilter.FilterKernel.Context[filters.length]; + for (int ii = 0; ii < filters.length; ii++) { + if (filters[ii].conditionFilter != null) { + conditionFilterContexts[ii] = filters[ii].conditionFilter.getContext(size); + } + } + resultsChunk = WritableBooleanChunk.makeWritableChunk(size); + // noinspection unchecked + filterChunks = new Chunk[filters.length][]; + for (int ii = 0; ii < filters.length; ii++) { + // noinspection unchecked + filterChunks[ii] = new Chunk[filters[ii].recorders.length]; + } + } + + @Override + public void close() { + SafeCloseableArray.close(conditionFilterContexts); + resultsChunk.close(); + // filterChunks are not owned by this context + } + } + + static class CountWhereBucketedContext extends BaseContext implements BucketedContext { + /** + * How many rows passed the filters for each destination. + */ + final WritableIntChunk countChunk; + /** + * How many previous rows passed the filters for each destination, only applies to modified rows. + */ + final WritableIntChunk previousCountChunk; + + private CountWhereBucketedContext(final int size, final CountFilter[] filters) { + super(size, filters); + countChunk = WritableIntChunk.makeWritableChunk(size); + previousCountChunk = WritableIntChunk.makeWritableChunk(size); + } + + @Override + public void close() { + super.close(); + countChunk.close(); + previousCountChunk.close(); + } + } + + static class CountWhereSingletonContext extends BaseContext implements SingletonContext { + private CountWhereSingletonContext(final int size, final CountFilter[] filters) { + super(size, filters); + } + } + + /** + * Construct a count aggregation operator that tests individual data values. + * + * @param resultName The name of the result column + */ + CountWhereOperator( + @NotNull final String resultName, + final WhereFilter[] filters, + final RecordingInternalOperator[] recorders, + final RecordingInternalOperator[][] filterRecorders) { + this.resultName = resultName; + this.resultColumnSource = new LongArraySource(); + + // Build a dummy table for use by generic WhereFilters and ConditionFilters + if (recorders.length == 0) { + chunkColumnSources = null; + chunkSourceTable = emptyTable(0); + } else { + chunkColumnSources = new ChunkColumnSource[recorders.length]; + final Map> columnSourceMap = new HashMap<>(); + for (int ii = 0; ii < recorders.length; ii++) { + final ColumnSource recorderSource = recorders[ii].getInputColumnSource(); + chunkColumnSources[ii] = ChunkColumnSource.make(recorderSource.getChunkType(), recorderSource.getType(), + recorderSource.getComponentType()); + columnSourceMap.put(recorders[ii].getInputColumnName(), chunkColumnSources[ii]); + } + chunkSourceTable = new QueryTable(RowSetFactory.empty().toTracking(), columnSourceMap); + } + + // Create the internal filters + final List filterList = new ArrayList<>(); + boolean forcedWhereFilter = false; + for (int fi = 0; fi < filters.length; fi++) { + final WhereFilter filter = filters[fi]; + final CountWhereOperator.CountFilter countFilter; + if (!forcedWhereFilter && filter instanceof ConditionFilter) { + final ConditionFilter conditionFilter = (ConditionFilter) filter; + if (conditionFilter.hasVirtualRowVariables()) { + throw new UnsupportedOperationException("AggCountWhere does not support refreshing filters"); + } + try { + countFilter = new CountWhereOperator.CountFilter( + conditionFilter.getFilter(chunkSourceTable, RowSetFactory.empty()), + filterRecorders[fi]); + } catch (final Exception e) { + throw new IllegalArgumentException("Error creating condition filter in CountWhereOperator", e); + } + } else if (!forcedWhereFilter && filter instanceof ExposesChunkFilter + && ((ExposesChunkFilter) filter).chunkFilter().isPresent()) { + final Optional chunkFilter = ((ExposesChunkFilter) filter).chunkFilter(); + countFilter = new CountWhereOperator.CountFilter(chunkFilter.get(), filterRecorders[fi]); + } else { + try (final SafeCloseable ignored = filter.beginOperation(chunkSourceTable)) { + countFilter = new CountWhereOperator.CountFilter(filter, filterRecorders[fi]); + } + forcedWhereFilter = true; + } + filterList.add(countFilter); + } + + this.updateChunkSourceTable = forcedWhereFilter; + this.filters = filterList.toArray(CountFilter[]::new); + this.recorders = recorders; + } + + /** + * Count the number of rows that passed the filters in the given chunk. + */ + private static int countChunk(final BooleanChunk values, final int start, final int len) { + int count = 0; + for (int ii = start; ii < start + len; ii++) { + if (values.get(ii)) { + count++; + } + } + return count; + } + + /** + * Build a RowSet from a boolean chunk, including only the indices that are true. + */ + private static WritableRowSet buildFromBooleanChunk(final BooleanChunk values, final int chunkSize) { + final RowSetBuilderSequential builder = RowSetFactory.builderSequential(); + for (int ii = 0; ii < chunkSize; ii++) { + if (values.get(ii)) { + builder.appendKey(ii); + } + } + return builder.build(); + } + + /** + * Apply the filters to the data in the chunk, returning the number of rows that passed the filters. If + * {@code requiresPopulatedResultsChunk == true}, the results chunk will be populated with {@code true} for every + * row that passed the filters. + */ + private int applyFilters( + final BaseContext ctx, + final int chunkSize, + final boolean requiresPopulatedResultsChunk) { + + boolean initialized = false; + WritableRowSet remainingRows = null; + final RowSet flatRowSet = RowSetFactory.flat(chunkSize); + + int count = 0; + + // We must apply the filters in the order they were given. + for (int fi = 0; fi < filters.length; fi++) { + final CountFilter filter = filters[fi]; + final Chunk[] valueChunks = ctx.filterChunks[fi]; + final ConditionFilter.FilterKernel.Context conditionalFilterContext = ctx.conditionFilterContexts[fi]; + + if (filter.chunkFilter != null) { + if (!initialized) { + // Chunk filters only have one input. + count = filter.chunkFilter.filter(valueChunks[0], ctx.resultsChunk); + initialized = true; + } else { + count = filter.chunkFilter.filterAnd(valueChunks[0], ctx.resultsChunk); + } + continue; + } else if (filter.conditionFilter != null) { + if (!initialized) { + count = filter.conditionFilter.filter(conditionalFilterContext, valueChunks, + chunkSize, ctx.resultsChunk); + initialized = true; + } else { + count = filter.conditionFilter.filterAnd(conditionalFilterContext, valueChunks, + chunkSize, ctx.resultsChunk); + } + continue; + } + + if (remainingRows == null) { + // This is the first WhereFilter to run, initialize the remainingRows RowSet + remainingRows = initialized + ? buildFromBooleanChunk(ctx.resultsChunk, chunkSize) + : RowSetFactory.flat(chunkSize); + } + try (final RowSet ignored = remainingRows) { + remainingRows = filter.whereFilter.filter(remainingRows, flatRowSet, chunkSourceTable, false); + } + initialized = true; + } + + try (final RowSet ignored = remainingRows; final RowSet ignored2 = flatRowSet) { + if (remainingRows != null) { + // WhereFilters were used, so gather the info from remainingRows + if (requiresPopulatedResultsChunk) { + ctx.resultsChunk.fillWithValue(0, chunkSize, false); + remainingRows.forAllRowKeyRanges( + (start, end) -> ctx.resultsChunk.fillWithValue((int) start, (int) end - (int) start + 1, + true)); + } + return remainingRows.intSize(); + } + } + return count; + } + + /** + * Using current data, update the provided context to contain the chunks from the recording operators and update the + * chunk source table if needed. + */ + private void updateChunkSources(final BaseContext ctx, final boolean usePrev) { + if (updateChunkSourceTable) { + for (int ii = 0; ii < chunkColumnSources.length; ii++) { + chunkColumnSources[ii].clear(); + final Chunk chunk = usePrev + ? recorders[ii].getPrevValueChunk() + : recorders[ii].getValueChunk(); + // ChunkColumnSource releases the chunks it acquires, so give it a copy. + final WritableChunk tmpValues = + (WritableChunk) chunk.slice(0, chunk.size()); + chunkColumnSources[ii].addChunk(tmpValues); + } + } + + // Grab the filter input chunks from the recorders + for (int fi = 0; fi < filters.length; fi++) { + for (int ri = 0; ri < filters[fi].recorders.length; ri++) { + final Chunk chunk = usePrev + ? filters[fi].recorders[ri].getPrevValueChunk() + : filters[fi].recorders[ri].getValueChunk(); + ctx.filterChunks[fi][ri] = chunk; + } + } + } + + /** + * Count the number of rows for each destination that pass the filters and store into the {@code destCountChunk}. + */ + private void doCountBucketed( + final CountWhereBucketedContext ctx, + final int chunkSize, + final IntChunk destinations, + final IntChunk startPositions, + final IntChunk length, + final WritableIntChunk destCountChunk, + final boolean usePrev) { + + updateChunkSources(ctx, usePrev); + + applyFilters(ctx, chunkSize, true); + + // fill the destination count chunk with the number of rows that passed the filter + for (int dest = 0; dest < startPositions.size(); dest++) { + final int start = startPositions.get(dest); + final int len = length.get(dest); + destCountChunk.set(dest, countChunk(ctx.resultsChunk, start, len)); + } + } + + /** + * Count the number of rows that pass the filters.. + */ + private int doCountSingleton( + final CountWhereSingletonContext ctx, + final int chunkSize, + final boolean usePrev) { + updateChunkSources(ctx, usePrev); + + return applyFilters(ctx, chunkSize, false); + } + + @Override + public void addChunk( + BucketedContext context, + Chunk values, + LongChunk inputRowKeys, + IntChunk destinations, + IntChunk startPositions, + IntChunk length, + WritableBooleanChunk stateModified) { + final CountWhereBucketedContext ctx = (CountWhereBucketedContext) context; + + // Compute the chunk size from the startPositions and length + final int chunkSize = startPositions.get(startPositions.size() - 1) + length.get(length.size() - 1); + + // Do the counting work and store the results in the count chunk + doCountBucketed(ctx, chunkSize, destinations, startPositions, length, ctx.countChunk, false); + + // Update the result column source with the counts for each destination + for (int ii = 0; ii < startPositions.size(); ++ii) { + final int startPosition = startPositions.get(ii); + final int count = ctx.countChunk.get(ii); + final long destination = destinations.get(startPosition); + resultColumnSource.set(destination, plusLong(resultColumnSource.getUnsafe(destination), count)); + stateModified.set(ii, true); + } + } + + @Override + public boolean addChunk( + SingletonContext context, + int chunkSize, + Chunk values, + LongChunk inputRowKeys, long destination) { + if (chunkSize == 0) { + return false; + } + + final CountWhereSingletonContext ctx = (CountWhereSingletonContext) context; + final int count = doCountSingleton(ctx, chunkSize, false); + resultColumnSource.set(destination, plusLong(resultColumnSource.getUnsafe(destination), count)); + return count > 0; + } + + @Override + public void removeChunk( + BucketedContext context, + Chunk values, + LongChunk inputRowKeys, + IntChunk destinations, + IntChunk startPositions, + IntChunk length, + WritableBooleanChunk stateModified) { + final CountWhereBucketedContext ctx = (CountWhereBucketedContext) context; + + // Compute the chunk size from the startPositions and length + final int chunkSize = startPositions.get(startPositions.size() - 1) + length.get(length.size() - 1); + + // Do the counting work and store the results in the count chunk + doCountBucketed(ctx, chunkSize, destinations, startPositions, length, ctx.countChunk, false); + + for (int ii = 0; ii < startPositions.size(); ++ii) { + final int count = ctx.countChunk.get(ii); + if (count > 0) { + final int startPosition = startPositions.get(ii); + final long destination = destinations.get(startPosition); + final long updatedCount = plusLong(resultColumnSource.getUnsafe(destination), -count); + + Assert.geqZero(updatedCount, "updatedCount"); + resultColumnSource.set(destination, updatedCount); + stateModified.set(ii, true); + } else { + stateModified.set(ii, false); + } + } + } + + @Override + public boolean removeChunk( + SingletonContext context, + int chunkSize, + Chunk values, + LongChunk inputRowKeys, + long destination) { + if (chunkSize == 0) { + return false; + } + + final CountWhereSingletonContext ctx = (CountWhereSingletonContext) context; + final int count = doCountSingleton(ctx, chunkSize, false); + if (count > 0) { + final long updatedCount = plusLong(resultColumnSource.getUnsafe(destination), -count); + Assert.geqZero(updatedCount, "updatedCount"); + resultColumnSource.set(destination, updatedCount); + return true; + } + return false; + } + + @Override + public void modifyChunk( + BucketedContext context, + Chunk previousValues, + Chunk newValues, + LongChunk postShiftRowKeys, + IntChunk destinations, + IntChunk startPositions, + IntChunk length, + WritableBooleanChunk stateModified) { + final CountWhereBucketedContext ctx = (CountWhereBucketedContext) context; + + // Compute the chunk size from the startPositions and length + final int chunkSize = startPositions.get(startPositions.size() - 1) + length.get(length.size() - 1); + + // Fill the context with previous chunk data + doCountBucketed(ctx, chunkSize, destinations, startPositions, length, ctx.previousCountChunk, true); + // Fill the context with current chunk data + doCountBucketed(ctx, chunkSize, destinations, startPositions, length, ctx.countChunk, false); + + for (int ii = 0; ii < startPositions.size(); ++ii) { + final int oldCount = ctx.previousCountChunk.get(ii); + final int newCount = ctx.countChunk.get(ii); + final int count = newCount - oldCount; + if (count != 0) { + final int startPosition = startPositions.get(ii); + final long destination = destinations.get(startPosition); + final long updatedCount = plusLong(resultColumnSource.getUnsafe(destination), count); + Assert.geqZero(updatedCount, "updatedCount"); + resultColumnSource.set(destination, updatedCount); + stateModified.set(ii, true); + } else { + stateModified.set(ii, false); + } + } + } + + @Override + public boolean modifyChunk( + SingletonContext context, + int chunkSize, + Chunk previousValues, + Chunk newValues, + LongChunk postShiftRowKeys, + long destination) { + if (chunkSize == 0) { + return false; + } + + final CountWhereSingletonContext ctx = (CountWhereSingletonContext) context; + final int prevCount = doCountSingleton(ctx, chunkSize, true); + final int newCount = doCountSingleton(ctx, chunkSize, false); + + final int count = newCount - prevCount; + if (count != 0) { + final long updatedCount = plusLong(resultColumnSource.getUnsafe(destination), count); + Assert.geqZero(updatedCount, "updatedCount"); + resultColumnSource.set(destination, updatedCount); + return true; + } + return false; + } + + @Override + public void ensureCapacity(long tableSize) { + resultColumnSource.ensureCapacity(tableSize, false); + } + + @Override + public Map> getResultColumns() { + return Collections.singletonMap(resultName, resultColumnSource); + } + + @Override + public void startTrackingPrevValues() { + resultColumnSource.startTrackingPrevValues(); + } + + @Override + public BucketedContext makeBucketedContext(int size) { + return new CountWhereBucketedContext(size, filters); + } + + @Override + public SingletonContext makeSingletonContext(int size) { + return new CountWhereSingletonContext(size, filters); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RecordingInternalOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RecordingInternalOperator.java new file mode 100644 index 00000000000..fadd17793a3 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/RecordingInternalOperator.java @@ -0,0 +1,162 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.by; + +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.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ColumnSource; + +import java.util.Collections; +import java.util.Map; + +/** + * Implements a generic recording operator that stores a chunk for retrieval by other operators. This must be added to + * the operator chain before the operator that will use the chunk. + *

+ * Recording operators are generally used to allow an operator to access more than one input column data. + */ +class RecordingInternalOperator implements IterativeChunkedAggregationOperator { + private Context currentContext = null; + private final String inputColumnName; + private final ColumnSource inputColumnSource; + + class Context implements SingletonContext, BucketedContext { + private Chunk valuesChunk; + private Chunk prevValuesChunk; + + private Context(int size) { + Assert.eqNull(currentContext, "currentContext.getValue()"); + currentContext = this; + } + + void setValues(Chunk values) { + valuesChunk = values; + } + + void setPrevValues(Chunk values) { + prevValuesChunk = values; + } + + @Override + public void close() { + Assert.eq(currentContext, "currentContext", this, "this"); + currentContext = null; + } + } + + public RecordingInternalOperator(String inputColumnName, ColumnSource inputColumnSource) { + this.inputColumnName = inputColumnName; + this.inputColumnSource = inputColumnSource; + } + + @Override + public void addChunk( + final BucketedContext context, Chunk values, + final LongChunk inputRowKeys, IntChunk destinations, + final IntChunk startPositions, IntChunk length, + final WritableBooleanChunk stateModified) { + ((Context) context).setValues(values); + } + + @Override + public boolean addChunk( + final SingletonContext context, + final int chunkSize, + final Chunk values, + final LongChunk inputRowKeys, long destination) { + ((Context) context).setValues(values); + return false; + } + + @Override + public void removeChunk( + final BucketedContext context, + final Chunk values, + final LongChunk inputRowKeys, + final IntChunk destinations, + final IntChunk startPositions, + final IntChunk length, + final WritableBooleanChunk stateModified) { + ((Context) context).setValues(values); + } + + @Override + public boolean removeChunk( + final SingletonContext context, + final int chunkSize, + final Chunk values, + final LongChunk inputRowKeys, + final long destination) { + ((Context) context).setValues(values); + return false; + } + + @Override + public void modifyChunk( + final BucketedContext context, + final Chunk previousValues, + final Chunk newValues, + final LongChunk postShiftRowKeys, + final IntChunk destinations, + final IntChunk startPositions, + final IntChunk length, + final WritableBooleanChunk stateModified) { + ((Context) context).setPrevValues(previousValues); + ((Context) context).setValues(newValues); + } + + @Override + public boolean modifyChunk( + final SingletonContext context, + final int chunkSize, + final Chunk previousValues, + final Chunk newValues, + final LongChunk postShiftRowKeys, + final long destination) { + ((Context) context).setPrevValues(previousValues); + ((Context) context).setValues(newValues); + return false; + } + + @Override + public void ensureCapacity(final long tableSize) {} + + @Override + public Map> getResultColumns() { + return Collections.emptyMap(); + } + + @Override + public void startTrackingPrevValues() {} + + public Chunk getValueChunk() { + return currentContext.valuesChunk; + } + + public Chunk getPrevValueChunk() { + return currentContext.prevValuesChunk; + } + + public String getInputColumnName() { + return inputColumnName; + } + + public ColumnSource getInputColumnSource() { + return inputColumnSource; + } + + @Override + public BucketedContext makeBucketedContext(final int size) { + return new Context(size); + } + + @Override + public SingletonContext makeSingletonContext(final int size) { + return new Context(size); + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkFilter.java new file mode 100644 index 00000000000..b32ba2935d2 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkFilter.java @@ -0,0 +1,67 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkFilter and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class ByteChunkFilter implements ChunkFilter { + public abstract boolean matches(byte value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final ByteChunk byteChunk = values.asByteChunk(); + final int len = byteChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(byteChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final ByteChunk byteChunk = values.asByteChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(byteChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final ByteChunk byteChunk = values.asByteChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(byteChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkMatchFilterFactory.java index 24ddc5b788d..72e5226984e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteChunkMatchFilterFactory.java @@ -7,23 +7,20 @@ // @formatter:off package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TByteHashSet; /** * Creates chunk filters for byte values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class ByteChunkMatchFilterFactory { private ByteChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.ByteChunkFilter makeFilter(boolean invertMatch, byte... values) { + public static ByteChunkFilter makeFilter(boolean invertMatch, byte... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueByteChunkFilter(values[0]); @@ -49,7 +46,7 @@ public static ChunkFilter.ByteChunkFilter makeFilter(boolean invertMatch, byte.. } } - private static class SingleValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class SingleValueByteChunkFilter extends ByteChunkFilter { private final byte value; private SingleValueByteChunkFilter(byte value) { @@ -57,18 +54,12 @@ private SingleValueByteChunkFilter(byte value) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return value == this.value; } } - private static class InverseSingleValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class InverseSingleValueByteChunkFilter extends ByteChunkFilter { private final byte value; private InverseSingleValueByteChunkFilter(byte value) { @@ -76,18 +67,12 @@ private InverseSingleValueByteChunkFilter(byte value) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return value != this.value; } } - private static class TwoValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class TwoValueByteChunkFilter extends ByteChunkFilter { private final byte value1; private final byte value2; @@ -97,19 +82,12 @@ private TwoValueByteChunkFilter(byte value1, byte value2) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class InverseTwoValueByteChunkFilter extends ByteChunkFilter { private final byte value1; private final byte value2; @@ -119,19 +97,12 @@ private InverseTwoValueByteChunkFilter(byte value1, byte value2) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return value != value1 && value != value2; } } - private static class ThreeValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class ThreeValueByteChunkFilter extends ByteChunkFilter { private final byte value1; private final byte value2; private final byte value3; @@ -143,19 +114,12 @@ private ThreeValueByteChunkFilter(byte value1, byte value2, byte value3) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class InverseThreeValueByteChunkFilter extends ByteChunkFilter { private final byte value1; private final byte value2; private final byte value3; @@ -167,19 +131,12 @@ private InverseThreeValueByteChunkFilter(byte value1, byte value2, byte value3) } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class MultiValueByteChunkFilter extends ByteChunkFilter { private final TByteHashSet values; private MultiValueByteChunkFilter(byte... values) { @@ -187,19 +144,12 @@ private MultiValueByteChunkFilter(byte... values) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return this.values.contains(value); } } - private static class InverseMultiValueByteChunkFilter implements ChunkFilter.ByteChunkFilter { + private final static class InverseMultiValueByteChunkFilter extends ByteChunkFilter { private final TByteHashSet values; private InverseMultiValueByteChunkFilter(byte... values) { @@ -207,15 +157,8 @@ private InverseMultiValueByteChunkFilter(byte... values) { } @Override - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(byte value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteRangeComparator.java index 4390d3268f6..1575cb4f661 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ByteRangeComparator.java @@ -8,14 +8,11 @@ package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.ByteComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class ByteRangeComparator { private ByteRangeComparator() {} // static use only - private abstract static class ByteByteFilter implements ChunkFilter.ByteChunkFilter { + private abstract static class ByteByteFilter extends ByteChunkFilter { final byte lower; final byte upper; @@ -23,80 +20,53 @@ private abstract static class ByteByteFilter implements ChunkFilter.ByteChunkFil this.lower = lower; this.upper = upper; } - - abstract public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results); } - static class ByteByteInclusiveInclusiveFilter extends ByteByteFilter { + final static class ByteByteInclusiveInclusiveFilter extends ByteByteFilter { private ByteByteInclusiveInclusiveFilter(byte lower, byte upper) { super(lower, upper); } - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte value = values.get(ii); - if (ByteComparisons.geq(value, lower) && ByteComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(byte value) { + return ByteComparisons.geq(value, lower) && ByteComparisons.leq(value, upper); } } - static class ByteByteInclusiveExclusiveFilter extends ByteByteFilter { + final static class ByteByteInclusiveExclusiveFilter extends ByteByteFilter { private ByteByteInclusiveExclusiveFilter(byte lower, byte upper) { super(lower, upper); } - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte value = values.get(ii); - if (ByteComparisons.geq(value, lower) && ByteComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(byte value) { + return ByteComparisons.geq(value, lower) && ByteComparisons.lt(value, upper); } } - static class ByteByteExclusiveInclusiveFilter extends ByteByteFilter { + final static class ByteByteExclusiveInclusiveFilter extends ByteByteFilter { private ByteByteExclusiveInclusiveFilter(byte lower, byte upper) { super(lower, upper); } - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte value = values.get(ii); - if (ByteComparisons.gt(value, lower) && ByteComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(byte value) { + return ByteComparisons.gt(value, lower) && ByteComparisons.leq(value, upper); } } - static class ByteByteExclusiveExclusiveFilter extends ByteByteFilter { + final static class ByteByteExclusiveExclusiveFilter extends ByteByteFilter { private ByteByteExclusiveExclusiveFilter(byte lower, byte upper) { super(lower, upper); } - public void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final byte value = values.get(ii); - if (ByteComparisons.gt(value, lower) && ByteComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(byte value) { + return ByteComparisons.gt(value, lower) && ByteComparisons.lt(value, upper); } } - public static ChunkFilter.ByteChunkFilter makeByteFilter(byte lower, byte upper, boolean lowerInclusive, + public static ByteChunkFilter makeByteFilter(byte lower, byte upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkFilter.java new file mode 100644 index 00000000000..833963308a9 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkFilter.java @@ -0,0 +1,63 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class CharChunkFilter implements ChunkFilter { + public abstract boolean matches(char value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final CharChunk charChunk = values.asCharChunk(); + final int len = charChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(charChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final CharChunk charChunk = values.asCharChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(charChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final CharChunk charChunk = values.asCharChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(charChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkMatchFilterFactory.java index 1a432399764..11f1dce9ae6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkMatchFilterFactory.java @@ -3,23 +3,20 @@ // package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TCharHashSet; /** * Creates chunk filters for char values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class CharChunkMatchFilterFactory { private CharChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.CharChunkFilter makeFilter(boolean invertMatch, char... values) { + public static CharChunkFilter makeFilter(boolean invertMatch, char... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueCharChunkFilter(values[0]); @@ -45,7 +42,7 @@ public static ChunkFilter.CharChunkFilter makeFilter(boolean invertMatch, char.. } } - private static class SingleValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class SingleValueCharChunkFilter extends CharChunkFilter { private final char value; private SingleValueCharChunkFilter(char value) { @@ -53,18 +50,12 @@ private SingleValueCharChunkFilter(char value) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return value == this.value; } } - private static class InverseSingleValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class InverseSingleValueCharChunkFilter extends CharChunkFilter { private final char value; private InverseSingleValueCharChunkFilter(char value) { @@ -72,18 +63,12 @@ private InverseSingleValueCharChunkFilter(char value) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return value != this.value; } } - private static class TwoValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class TwoValueCharChunkFilter extends CharChunkFilter { private final char value1; private final char value2; @@ -93,19 +78,12 @@ private TwoValueCharChunkFilter(char value1, char value2) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class InverseTwoValueCharChunkFilter extends CharChunkFilter { private final char value1; private final char value2; @@ -115,19 +93,12 @@ private InverseTwoValueCharChunkFilter(char value1, char value2) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return value != value1 && value != value2; } } - private static class ThreeValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class ThreeValueCharChunkFilter extends CharChunkFilter { private final char value1; private final char value2; private final char value3; @@ -139,19 +110,12 @@ private ThreeValueCharChunkFilter(char value1, char value2, char value3) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class InverseThreeValueCharChunkFilter extends CharChunkFilter { private final char value1; private final char value2; private final char value3; @@ -163,19 +127,12 @@ private InverseThreeValueCharChunkFilter(char value1, char value2, char value3) } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class MultiValueCharChunkFilter extends CharChunkFilter { private final TCharHashSet values; private MultiValueCharChunkFilter(char... values) { @@ -183,19 +140,12 @@ private MultiValueCharChunkFilter(char... values) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return this.values.contains(value); } } - private static class InverseMultiValueCharChunkFilter implements ChunkFilter.CharChunkFilter { + private final static class InverseMultiValueCharChunkFilter extends CharChunkFilter { private final TCharHashSet values; private InverseMultiValueCharChunkFilter(char... values) { @@ -203,15 +153,8 @@ private InverseMultiValueCharChunkFilter(char... values) { } @Override - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(char value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java index 6c75bd5a946..3e3a01027f6 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java @@ -4,14 +4,11 @@ package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.CharComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class CharRangeComparator { private CharRangeComparator() {} // static use only - private abstract static class CharCharFilter implements ChunkFilter.CharChunkFilter { + private abstract static class CharCharFilter extends CharChunkFilter { final char lower; final char upper; @@ -19,80 +16,53 @@ private abstract static class CharCharFilter implements ChunkFilter.CharChunkFil this.lower = lower; this.upper = upper; } - - abstract public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results); } - static class CharCharInclusiveInclusiveFilter extends CharCharFilter { + final static class CharCharInclusiveInclusiveFilter extends CharCharFilter { private CharCharInclusiveInclusiveFilter(char lower, char upper) { super(lower, upper); } - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char value = values.get(ii); - if (CharComparisons.geq(value, lower) && CharComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(char value) { + return CharComparisons.geq(value, lower) && CharComparisons.leq(value, upper); } } - static class CharCharInclusiveExclusiveFilter extends CharCharFilter { + final static class CharCharInclusiveExclusiveFilter extends CharCharFilter { private CharCharInclusiveExclusiveFilter(char lower, char upper) { super(lower, upper); } - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char value = values.get(ii); - if (CharComparisons.geq(value, lower) && CharComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(char value) { + return CharComparisons.geq(value, lower) && CharComparisons.lt(value, upper); } } - static class CharCharExclusiveInclusiveFilter extends CharCharFilter { + final static class CharCharExclusiveInclusiveFilter extends CharCharFilter { private CharCharExclusiveInclusiveFilter(char lower, char upper) { super(lower, upper); } - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char value = values.get(ii); - if (CharComparisons.gt(value, lower) && CharComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(char value) { + return CharComparisons.gt(value, lower) && CharComparisons.leq(value, upper); } } - static class CharCharExclusiveExclusiveFilter extends CharCharFilter { + final static class CharCharExclusiveExclusiveFilter extends CharCharFilter { private CharCharExclusiveExclusiveFilter(char lower, char upper) { super(lower, upper); } - public void filter(CharChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final char value = values.get(ii); - if (CharComparisons.gt(value, lower) && CharComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(char value) { + return CharComparisons.gt(value, lower) && CharComparisons.lt(value, upper); } } - public static ChunkFilter.CharChunkFilter makeCharFilter(char lower, char upper, boolean lowerInclusive, + public static CharChunkFilter makeCharFilter(char lower, char upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ChunkFilter.java index 562d50b1bba..27082102b82 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ChunkFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ChunkFilter.java @@ -15,107 +15,95 @@ public interface ChunkFilter { /** * Filter a chunk of values, setting parallel values in results to "true" or "false". - * + *

* The results chunk must have capacity at least as large as values.size(); and the result size will be set to * values.size() on return. * * @param values the values to filter - * @param results a boolean chunk with true values for items that match the filter, and false otherwise + * @param results a chunk with the key values where the result of the filter is true */ void filter(Chunk values, LongChunk keys, WritableLongChunk results); - interface CharChunkFilter extends ChunkFilter { - void filter(CharChunk values, LongChunk keys, - WritableLongChunk results); - - default void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - filter(values.asCharChunk(), keys, results); - } - } + /** + * Filter a chunk of values, setting parallel values in {@code results} to the output of the filter. + * + * @param values the values to filter + * @param results a boolean chunk containing the result of the filter + * + * @return the number of values that are {@code true} in {@code results} after the filter is applied. + */ + int filter(Chunk values, WritableBooleanChunk results); - interface ByteChunkFilter extends ChunkFilter { - void filter(ByteChunk values, LongChunk keys, - WritableLongChunk results); + /** + * Filter a chunk of values, setting parallel values in {@code results} to {@code false} when the filter result is + * {@code false}. The filter will not be evaluated for values that are already {@code false} in the results chunk. + *

+ * To use this method effectively, the results chunk should be initialized by a call to + * {@link #filter(Chunk, WritableBooleanChunk)} or by setting all values {@code true} before the first call. + * Successive calls will have the effect of AND'ing the filter results with existing results. + * + * @param values the values to filter + * @param results a boolean chunk containing the result of the filter + * + * @return the number of values that are {@code true} in {@code results} after the filter is applied. + */ + int filterAnd(Chunk values, WritableBooleanChunk results); - default void filter(Chunk values, LongChunk keys, + /** + * A filter that always returns false. + */ + ChunkFilter FALSE_FILTER_INSTANCE = new ChunkFilter() { + @Override + public void filter(Chunk values, LongChunk keys, WritableLongChunk results) { - filter(values.asByteChunk(), keys, results); + results.setSize(0); } - } - - interface ShortChunkFilter extends ChunkFilter { - void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results); - default void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - filter(values.asShortChunk(), keys, results); + @Override + public int filter(final Chunk values, final WritableBooleanChunk results) { + final int len = values.size(); + results.fillWithValue(0, len, false); + return 0; } - } - - interface IntChunkFilter extends ChunkFilter { - void filter(IntChunk values, LongChunk keys, - WritableLongChunk results); - default void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - filter(values.asIntChunk(), keys, results); - } - } - - interface LongChunkFilter extends ChunkFilter { - void filter(LongChunk values, LongChunk keys, - WritableLongChunk results); - - default void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - filter(values.asLongChunk(), keys, results); + @Override + public int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final int len = values.size(); + // Count the values that changed from true to false + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + results.set(ii, false); + count += result ? 1 : 0; + } + return count; } - } - - interface FloatChunkFilter extends ChunkFilter { - void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results); + }; - default void filter(Chunk values, LongChunk keys, + /** + * A filter that always returns true. + */ + ChunkFilter TRUE_FILTER_INSTANCE = new ChunkFilter() { + @Override + public void filter(Chunk values, LongChunk keys, WritableLongChunk results) { - filter(values.asFloatChunk(), keys, results); + results.setSize(values.size()); + results.copyFromChunk(keys, 0, 0, keys.size()); } - } - - interface DoubleChunkFilter extends ChunkFilter { - void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results); - default void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - filter(values.asDoubleChunk(), keys, results); + @Override + public int filter(final Chunk values, final WritableBooleanChunk results) { + final int len = values.size(); + results.fillWithValue(0, len, true); + return len; } - } - - interface ObjectChunkFilter extends ChunkFilter { - void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results); - default void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - filter(values.asObjectChunk(), keys, results); + @Override + public int filterAnd(final Chunk values, final WritableBooleanChunk results) { + // No values were set to false + return 0; } - } - - /** - * A filter that always returns false. - */ - ChunkFilter FALSE_FILTER_INSTANCE = (values, keys, results) -> results.setSize(0); - - /** - * A filter that always returns true. - */ - ChunkFilter TRUE_FILTER_INSTANCE = (values, keys, results) -> { - results.setSize(values.size()); - results.copyFromChunk(keys, 0, 0, keys.size()); }; /** diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkFilter.java new file mode 100644 index 00000000000..d38d66a3016 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkFilter.java @@ -0,0 +1,67 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkFilter and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class DoubleChunkFilter implements ChunkFilter { + public abstract boolean matches(double value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final DoubleChunk doubleChunk = values.asDoubleChunk(); + final int len = doubleChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(doubleChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final DoubleChunk doubleChunk = values.asDoubleChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(doubleChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final DoubleChunk doubleChunk = values.asDoubleChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(doubleChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkMatchFilterFactory.java index 1068c96eed2..16e5549f626 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleChunkMatchFilterFactory.java @@ -7,23 +7,20 @@ // @formatter:off package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TDoubleHashSet; /** * Creates chunk filters for double values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class DoubleChunkMatchFilterFactory { private DoubleChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.DoubleChunkFilter makeFilter(boolean invertMatch, double... values) { + public static DoubleChunkFilter makeFilter(boolean invertMatch, double... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueDoubleChunkFilter(values[0]); @@ -49,7 +46,7 @@ public static ChunkFilter.DoubleChunkFilter makeFilter(boolean invertMatch, doub } } - private static class SingleValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class SingleValueDoubleChunkFilter extends DoubleChunkFilter { private final double value; private SingleValueDoubleChunkFilter(double value) { @@ -57,18 +54,12 @@ private SingleValueDoubleChunkFilter(double value) { } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return value == this.value; } } - private static class InverseSingleValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class InverseSingleValueDoubleChunkFilter extends DoubleChunkFilter { private final double value; private InverseSingleValueDoubleChunkFilter(double value) { @@ -76,18 +67,12 @@ private InverseSingleValueDoubleChunkFilter(double value) { } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return value != this.value; } } - private static class TwoValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class TwoValueDoubleChunkFilter extends DoubleChunkFilter { private final double value1; private final double value2; @@ -97,19 +82,12 @@ private TwoValueDoubleChunkFilter(double value1, double value2) { } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class InverseTwoValueDoubleChunkFilter extends DoubleChunkFilter { private final double value1; private final double value2; @@ -119,19 +97,12 @@ private InverseTwoValueDoubleChunkFilter(double value1, double value2) { } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return value != value1 && value != value2; } } - private static class ThreeValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class ThreeValueDoubleChunkFilter extends DoubleChunkFilter { private final double value1; private final double value2; private final double value3; @@ -143,19 +114,12 @@ private ThreeValueDoubleChunkFilter(double value1, double value2, double value3) } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class InverseThreeValueDoubleChunkFilter extends DoubleChunkFilter { private final double value1; private final double value2; private final double value3; @@ -167,19 +131,12 @@ private InverseThreeValueDoubleChunkFilter(double value1, double value2, double } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class MultiValueDoubleChunkFilter extends DoubleChunkFilter { private final TDoubleHashSet values; private MultiValueDoubleChunkFilter(double... values) { @@ -187,19 +144,12 @@ private MultiValueDoubleChunkFilter(double... values) { } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return this.values.contains(value); } } - private static class InverseMultiValueDoubleChunkFilter implements ChunkFilter.DoubleChunkFilter { + private final static class InverseMultiValueDoubleChunkFilter extends DoubleChunkFilter { private final TDoubleHashSet values; private InverseMultiValueDoubleChunkFilter(double... values) { @@ -207,15 +157,8 @@ private InverseMultiValueDoubleChunkFilter(double... values) { } @Override - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(double value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleRangeComparator.java index e250e127181..669be87ed72 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/DoubleRangeComparator.java @@ -8,14 +8,11 @@ package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.DoubleComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class DoubleRangeComparator { private DoubleRangeComparator() {} // static use only - private abstract static class DoubleDoubleFilter implements ChunkFilter.DoubleChunkFilter { + private abstract static class DoubleDoubleFilter extends DoubleChunkFilter { final double lower; final double upper; @@ -23,80 +20,53 @@ private abstract static class DoubleDoubleFilter implements ChunkFilter.DoubleCh this.lower = lower; this.upper = upper; } - - abstract public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results); } - static class DoubleDoubleInclusiveInclusiveFilter extends DoubleDoubleFilter { + private final static class DoubleDoubleInclusiveInclusiveFilter extends DoubleDoubleFilter { private DoubleDoubleInclusiveInclusiveFilter(double lower, double upper) { super(lower, upper); } - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double value = values.get(ii); - if (DoubleComparisons.geq(value, lower) && DoubleComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(double value) { + return DoubleComparisons.geq(value, lower) && DoubleComparisons.leq(value, upper); } } - static class DoubleDoubleInclusiveExclusiveFilter extends DoubleDoubleFilter { + private final static class DoubleDoubleInclusiveExclusiveFilter extends DoubleDoubleFilter { private DoubleDoubleInclusiveExclusiveFilter(double lower, double upper) { super(lower, upper); } - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double value = values.get(ii); - if (DoubleComparisons.geq(value, lower) && DoubleComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(double value) { + return DoubleComparisons.geq(value, lower) && DoubleComparisons.lt(value, upper); } } - static class DoubleDoubleExclusiveInclusiveFilter extends DoubleDoubleFilter { + private final static class DoubleDoubleExclusiveInclusiveFilter extends DoubleDoubleFilter { private DoubleDoubleExclusiveInclusiveFilter(double lower, double upper) { super(lower, upper); } - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double value = values.get(ii); - if (DoubleComparisons.gt(value, lower) && DoubleComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(double value) { + return DoubleComparisons.gt(value, lower) && DoubleComparisons.leq(value, upper); } } - static class DoubleDoubleExclusiveExclusiveFilter extends DoubleDoubleFilter { + private final static class DoubleDoubleExclusiveExclusiveFilter extends DoubleDoubleFilter { private DoubleDoubleExclusiveExclusiveFilter(double lower, double upper) { super(lower, upper); } - public void filter(DoubleChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final double value = values.get(ii); - if (DoubleComparisons.gt(value, lower) && DoubleComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(double value) { + return DoubleComparisons.gt(value, lower) && DoubleComparisons.lt(value, upper); } } - public static ChunkFilter.DoubleChunkFilter makeDoubleFilter(double lower, double upper, boolean lowerInclusive, + public static DoubleChunkFilter makeDoubleFilter(double lower, double upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkFilter.java new file mode 100644 index 00000000000..b6b85106a42 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkFilter.java @@ -0,0 +1,67 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkFilter and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class FloatChunkFilter implements ChunkFilter { + public abstract boolean matches(float value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final FloatChunk floatChunk = values.asFloatChunk(); + final int len = floatChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(floatChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final FloatChunk floatChunk = values.asFloatChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(floatChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final FloatChunk floatChunk = values.asFloatChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(floatChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkMatchFilterFactory.java index e98d9f6b85f..9fa345b5278 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatChunkMatchFilterFactory.java @@ -7,23 +7,20 @@ // @formatter:off package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TFloatHashSet; /** * Creates chunk filters for float values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class FloatChunkMatchFilterFactory { private FloatChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.FloatChunkFilter makeFilter(boolean invertMatch, float... values) { + public static FloatChunkFilter makeFilter(boolean invertMatch, float... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueFloatChunkFilter(values[0]); @@ -49,7 +46,7 @@ public static ChunkFilter.FloatChunkFilter makeFilter(boolean invertMatch, float } } - private static class SingleValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class SingleValueFloatChunkFilter extends FloatChunkFilter { private final float value; private SingleValueFloatChunkFilter(float value) { @@ -57,18 +54,12 @@ private SingleValueFloatChunkFilter(float value) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return value == this.value; } } - private static class InverseSingleValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class InverseSingleValueFloatChunkFilter extends FloatChunkFilter { private final float value; private InverseSingleValueFloatChunkFilter(float value) { @@ -76,18 +67,12 @@ private InverseSingleValueFloatChunkFilter(float value) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return value != this.value; } } - private static class TwoValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class TwoValueFloatChunkFilter extends FloatChunkFilter { private final float value1; private final float value2; @@ -97,19 +82,12 @@ private TwoValueFloatChunkFilter(float value1, float value2) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class InverseTwoValueFloatChunkFilter extends FloatChunkFilter { private final float value1; private final float value2; @@ -119,19 +97,12 @@ private InverseTwoValueFloatChunkFilter(float value1, float value2) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return value != value1 && value != value2; } } - private static class ThreeValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class ThreeValueFloatChunkFilter extends FloatChunkFilter { private final float value1; private final float value2; private final float value3; @@ -143,19 +114,12 @@ private ThreeValueFloatChunkFilter(float value1, float value2, float value3) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class InverseThreeValueFloatChunkFilter extends FloatChunkFilter { private final float value1; private final float value2; private final float value3; @@ -167,19 +131,12 @@ private InverseThreeValueFloatChunkFilter(float value1, float value2, float valu } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class MultiValueFloatChunkFilter extends FloatChunkFilter { private final TFloatHashSet values; private MultiValueFloatChunkFilter(float... values) { @@ -187,19 +144,12 @@ private MultiValueFloatChunkFilter(float... values) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return this.values.contains(value); } } - private static class InverseMultiValueFloatChunkFilter implements ChunkFilter.FloatChunkFilter { + private final static class InverseMultiValueFloatChunkFilter extends FloatChunkFilter { private final TFloatHashSet values; private InverseMultiValueFloatChunkFilter(float... values) { @@ -207,15 +157,8 @@ private InverseMultiValueFloatChunkFilter(float... values) { } @Override - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(float value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatRangeComparator.java index 2d9027c446e..caa3f084b1f 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/FloatRangeComparator.java @@ -4,14 +4,11 @@ package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.FloatComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class FloatRangeComparator { private FloatRangeComparator() {} // static use only - private abstract static class FloatFloatFilter implements ChunkFilter.FloatChunkFilter { + private abstract static class FloatFloatFilter extends FloatChunkFilter { final float lower; final float upper; @@ -19,80 +16,53 @@ private abstract static class FloatFloatFilter implements ChunkFilter.FloatChunk this.lower = lower; this.upper = upper; } - - abstract public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results); } - static class FloatDoubleInclusiveInclusiveFilter extends FloatFloatFilter { + private final static class FloatDoubleInclusiveInclusiveFilter extends FloatFloatFilter { private FloatDoubleInclusiveInclusiveFilter(float lower, float upper) { super(lower, upper); } - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float value = values.get(ii); - if (FloatComparisons.geq(value, lower) && FloatComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(float value) { + return FloatComparisons.geq(value, lower) && FloatComparisons.leq(value, upper); } } - static class FloatDoubleInclusiveExclusiveFilter extends FloatFloatFilter { + private final static class FloatDoubleInclusiveExclusiveFilter extends FloatFloatFilter { private FloatDoubleInclusiveExclusiveFilter(float lower, float upper) { super(lower, upper); } - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float value = values.get(ii); - if (FloatComparisons.geq(value, lower) && FloatComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(float value) { + return FloatComparisons.geq(value, lower) && FloatComparisons.lt(value, upper); } } - static class FloatDoubleExclusiveInclusiveFilter extends FloatFloatFilter { + private final static class FloatDoubleExclusiveInclusiveFilter extends FloatFloatFilter { private FloatDoubleExclusiveInclusiveFilter(float lower, float upper) { super(lower, upper); } - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float value = values.get(ii); - if (FloatComparisons.gt(value, lower) && FloatComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(float value) { + return FloatComparisons.gt(value, lower) && FloatComparisons.leq(value, upper); } } - static class FloatDoubleExclusiveExclusiveFilter extends FloatFloatFilter { + private final static class FloatDoubleExclusiveExclusiveFilter extends FloatFloatFilter { private FloatDoubleExclusiveExclusiveFilter(float lower, float upper) { super(lower, upper); } - public void filter(FloatChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final float value = values.get(ii); - if (FloatComparisons.gt(value, lower) && FloatComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(float value) { + return FloatComparisons.gt(value, lower) && FloatComparisons.lt(value, upper); } } - public static ChunkFilter.FloatChunkFilter makeFloatFilter(float lower, float upper, boolean lowerInclusive, + public static FloatChunkFilter makeFloatFilter(float lower, float upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkFilter.java new file mode 100644 index 00000000000..c8dd051e53a --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkFilter.java @@ -0,0 +1,67 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkFilter and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class IntChunkFilter implements ChunkFilter { + public abstract boolean matches(int value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final IntChunk intChunk = values.asIntChunk(); + final int len = intChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(intChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final IntChunk intChunk = values.asIntChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(intChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final IntChunk intChunk = values.asIntChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(intChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkMatchFilterFactory.java index 2199c7f9382..6af8527b13e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntChunkMatchFilterFactory.java @@ -7,23 +7,20 @@ // @formatter:off package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TIntHashSet; /** * Creates chunk filters for int values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class IntChunkMatchFilterFactory { private IntChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.IntChunkFilter makeFilter(boolean invertMatch, int... values) { + public static IntChunkFilter makeFilter(boolean invertMatch, int... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueIntChunkFilter(values[0]); @@ -49,7 +46,7 @@ public static ChunkFilter.IntChunkFilter makeFilter(boolean invertMatch, int... } } - private static class SingleValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class SingleValueIntChunkFilter extends IntChunkFilter { private final int value; private SingleValueIntChunkFilter(int value) { @@ -57,18 +54,12 @@ private SingleValueIntChunkFilter(int value) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return value == this.value; } } - private static class InverseSingleValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class InverseSingleValueIntChunkFilter extends IntChunkFilter { private final int value; private InverseSingleValueIntChunkFilter(int value) { @@ -76,18 +67,12 @@ private InverseSingleValueIntChunkFilter(int value) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return value != this.value; } } - private static class TwoValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class TwoValueIntChunkFilter extends IntChunkFilter { private final int value1; private final int value2; @@ -97,19 +82,12 @@ private TwoValueIntChunkFilter(int value1, int value2) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class InverseTwoValueIntChunkFilter extends IntChunkFilter { private final int value1; private final int value2; @@ -119,19 +97,12 @@ private InverseTwoValueIntChunkFilter(int value1, int value2) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return value != value1 && value != value2; } } - private static class ThreeValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class ThreeValueIntChunkFilter extends IntChunkFilter { private final int value1; private final int value2; private final int value3; @@ -143,19 +114,12 @@ private ThreeValueIntChunkFilter(int value1, int value2, int value3) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class InverseThreeValueIntChunkFilter extends IntChunkFilter { private final int value1; private final int value2; private final int value3; @@ -167,19 +131,12 @@ private InverseThreeValueIntChunkFilter(int value1, int value2, int value3) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class MultiValueIntChunkFilter extends IntChunkFilter { private final TIntHashSet values; private MultiValueIntChunkFilter(int... values) { @@ -187,19 +144,12 @@ private MultiValueIntChunkFilter(int... values) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return this.values.contains(value); } } - private static class InverseMultiValueIntChunkFilter implements ChunkFilter.IntChunkFilter { + private final static class InverseMultiValueIntChunkFilter extends IntChunkFilter { private final TIntHashSet values; private InverseMultiValueIntChunkFilter(int... values) { @@ -207,15 +157,8 @@ private InverseMultiValueIntChunkFilter(int... values) { } @Override - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(int value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntRangeComparator.java index 16fa808cd24..308bf04f69b 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/IntRangeComparator.java @@ -8,14 +8,11 @@ package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.IntComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class IntRangeComparator { private IntRangeComparator() {} // static use only - private abstract static class IntIntFilter implements ChunkFilter.IntChunkFilter { + private abstract static class IntIntFilter extends IntChunkFilter { final int lower; final int upper; @@ -23,80 +20,53 @@ private abstract static class IntIntFilter implements ChunkFilter.IntChunkFilter this.lower = lower; this.upper = upper; } - - abstract public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results); } - static class IntIntInclusiveInclusiveFilter extends IntIntFilter { + final static class IntIntInclusiveInclusiveFilter extends IntIntFilter { private IntIntInclusiveInclusiveFilter(int lower, int upper) { super(lower, upper); } - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int value = values.get(ii); - if (IntComparisons.geq(value, lower) && IntComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(int value) { + return IntComparisons.geq(value, lower) && IntComparisons.leq(value, upper); } } - static class IntIntInclusiveExclusiveFilter extends IntIntFilter { + final static class IntIntInclusiveExclusiveFilter extends IntIntFilter { private IntIntInclusiveExclusiveFilter(int lower, int upper) { super(lower, upper); } - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int value = values.get(ii); - if (IntComparisons.geq(value, lower) && IntComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(int value) { + return IntComparisons.geq(value, lower) && IntComparisons.lt(value, upper); } } - static class IntIntExclusiveInclusiveFilter extends IntIntFilter { + final static class IntIntExclusiveInclusiveFilter extends IntIntFilter { private IntIntExclusiveInclusiveFilter(int lower, int upper) { super(lower, upper); } - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int value = values.get(ii); - if (IntComparisons.gt(value, lower) && IntComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(int value) { + return IntComparisons.gt(value, lower) && IntComparisons.leq(value, upper); } } - static class IntIntExclusiveExclusiveFilter extends IntIntFilter { + final static class IntIntExclusiveExclusiveFilter extends IntIntFilter { private IntIntExclusiveExclusiveFilter(int lower, int upper) { super(lower, upper); } - public void filter(IntChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final int value = values.get(ii); - if (IntComparisons.gt(value, lower) && IntComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(int value) { + return IntComparisons.gt(value, lower) && IntComparisons.lt(value, upper); } } - public static ChunkFilter.IntChunkFilter makeIntFilter(int lower, int upper, boolean lowerInclusive, + public static IntChunkFilter makeIntFilter(int lower, int upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkFilter.java new file mode 100644 index 00000000000..4545fbb3fbf --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkFilter.java @@ -0,0 +1,67 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkFilter and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class LongChunkFilter implements ChunkFilter { + public abstract boolean matches(long value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final LongChunk longChunk = values.asLongChunk(); + final int len = longChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(longChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final LongChunk longChunk = values.asLongChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(longChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final LongChunk longChunk = values.asLongChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(longChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkMatchFilterFactory.java index e283ece3d82..74b932f8c94 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongChunkMatchFilterFactory.java @@ -7,23 +7,20 @@ // @formatter:off package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TLongHashSet; /** * Creates chunk filters for long values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class LongChunkMatchFilterFactory { private LongChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.LongChunkFilter makeFilter(boolean invertMatch, long... values) { + public static LongChunkFilter makeFilter(boolean invertMatch, long... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueLongChunkFilter(values[0]); @@ -49,7 +46,7 @@ public static ChunkFilter.LongChunkFilter makeFilter(boolean invertMatch, long.. } } - private static class SingleValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class SingleValueLongChunkFilter extends LongChunkFilter { private final long value; private SingleValueLongChunkFilter(long value) { @@ -57,18 +54,12 @@ private SingleValueLongChunkFilter(long value) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return value == this.value; } } - private static class InverseSingleValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class InverseSingleValueLongChunkFilter extends LongChunkFilter { private final long value; private InverseSingleValueLongChunkFilter(long value) { @@ -76,18 +67,12 @@ private InverseSingleValueLongChunkFilter(long value) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return value != this.value; } } - private static class TwoValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class TwoValueLongChunkFilter extends LongChunkFilter { private final long value1; private final long value2; @@ -97,19 +82,12 @@ private TwoValueLongChunkFilter(long value1, long value2) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class InverseTwoValueLongChunkFilter extends LongChunkFilter { private final long value1; private final long value2; @@ -119,19 +97,12 @@ private InverseTwoValueLongChunkFilter(long value1, long value2) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return value != value1 && value != value2; } } - private static class ThreeValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class ThreeValueLongChunkFilter extends LongChunkFilter { private final long value1; private final long value2; private final long value3; @@ -143,19 +114,12 @@ private ThreeValueLongChunkFilter(long value1, long value2, long value3) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class InverseThreeValueLongChunkFilter extends LongChunkFilter { private final long value1; private final long value2; private final long value3; @@ -167,19 +131,12 @@ private InverseThreeValueLongChunkFilter(long value1, long value2, long value3) } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class MultiValueLongChunkFilter extends LongChunkFilter { private final TLongHashSet values; private MultiValueLongChunkFilter(long... values) { @@ -187,19 +144,12 @@ private MultiValueLongChunkFilter(long... values) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return this.values.contains(value); } } - private static class InverseMultiValueLongChunkFilter implements ChunkFilter.LongChunkFilter { + private final static class InverseMultiValueLongChunkFilter extends LongChunkFilter { private final TLongHashSet values; private InverseMultiValueLongChunkFilter(long... values) { @@ -207,15 +157,8 @@ private InverseMultiValueLongChunkFilter(long... values) { } @Override - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(long value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongRangeComparator.java index b150b80e880..bf910b99194 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/LongRangeComparator.java @@ -1,17 +1,18 @@ // // Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending // +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharRangeComparator and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.LongComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class LongRangeComparator { private LongRangeComparator() {} // static use only - private abstract static class LongLongFilter implements ChunkFilter.LongChunkFilter { + private abstract static class LongLongFilter extends LongChunkFilter { final long lower; final long upper; @@ -19,80 +20,53 @@ private abstract static class LongLongFilter implements ChunkFilter.LongChunkFil this.lower = lower; this.upper = upper; } - - abstract public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results); } - static class LongLongInclusiveInclusiveFilter extends LongLongFilter { + final static class LongLongInclusiveInclusiveFilter extends LongLongFilter { private LongLongInclusiveInclusiveFilter(long lower, long upper) { super(lower, upper); } - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long value = values.get(ii); - if (LongComparisons.geq(value, lower) && LongComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(long value) { + return LongComparisons.geq(value, lower) && LongComparisons.leq(value, upper); } } - static class LongLongInclusiveExclusiveFilter extends LongLongFilter { + final static class LongLongInclusiveExclusiveFilter extends LongLongFilter { private LongLongInclusiveExclusiveFilter(long lower, long upper) { super(lower, upper); } - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long value = values.get(ii); - if (LongComparisons.geq(value, lower) && LongComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(long value) { + return LongComparisons.geq(value, lower) && LongComparisons.lt(value, upper); } } - static class LongLongExclusiveInclusiveFilter extends LongLongFilter { + final static class LongLongExclusiveInclusiveFilter extends LongLongFilter { private LongLongExclusiveInclusiveFilter(long lower, long upper) { super(lower, upper); } - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long value = values.get(ii); - if (LongComparisons.gt(value, lower) && LongComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(long value) { + return LongComparisons.gt(value, lower) && LongComparisons.leq(value, upper); } } - static class LongLongExclusiveExclusiveFilter extends LongLongFilter { + final static class LongLongExclusiveExclusiveFilter extends LongLongFilter { private LongLongExclusiveExclusiveFilter(long lower, long upper) { super(lower, upper); } - public void filter(LongChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final long value = values.get(ii); - if (LongComparisons.gt(value, lower) && LongComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(long value) { + return LongComparisons.gt(value, lower) && LongComparisons.lt(value, upper); } } - public static ChunkFilter.LongChunkFilter makeLongFilter(long lower, long upper, boolean lowerInclusive, + public static LongChunkFilter makeLongFilter(long lower, long upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkFilter.java new file mode 100644 index 00000000000..700c8653ac6 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkFilter.java @@ -0,0 +1,63 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class ObjectChunkFilter implements ChunkFilter { + public abstract boolean matches(T value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final ObjectChunk objectChunk = values.asObjectChunk(); + final int len = objectChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(objectChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final ObjectChunk objectChunk = values.asObjectChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(objectChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final ObjectChunk objectChunk = values.asObjectChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(objectChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkMatchFilterFactory.java index 0d32524fc33..4cafb621563 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ObjectChunkMatchFilterFactory.java @@ -3,10 +3,6 @@ // package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; - import java.util.Arrays; import java.util.HashSet; import java.util.Objects; @@ -14,7 +10,8 @@ public class ObjectChunkMatchFilterFactory { private ObjectChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.ObjectChunkFilter makeFilter(boolean invert, Object... values) { + @SuppressWarnings("rawtypes") + public static ObjectChunkFilter makeFilter(boolean invert, Object... values) { if (invert) { if (values.length == 1) { return new InverseSingleValueObjectChunkFilter(values[0]); @@ -40,7 +37,7 @@ public static ChunkFilter.ObjectChunkFilter makeFilter(boolean invert, Object... } } - private static class SingleValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class SingleValueObjectChunkFilter extends ObjectChunkFilter { private final Object value; private SingleValueObjectChunkFilter(Object value) { @@ -48,18 +45,12 @@ private SingleValueObjectChunkFilter(Object value) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (Objects.equals(values.get(ii), value)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return Objects.equals(value, this.value); } } - private static class InverseSingleValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class InverseSingleValueObjectChunkFilter extends ObjectChunkFilter { private final Object value; private InverseSingleValueObjectChunkFilter(Object value) { @@ -67,18 +58,12 @@ private InverseSingleValueObjectChunkFilter(Object value) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (!Objects.equals(values.get(ii), value)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return !Objects.equals(value, this.value); } } - private static class TwoValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class TwoValueObjectChunkFilter extends ObjectChunkFilter { private final Object value1; private final Object value2; @@ -88,19 +73,12 @@ private TwoValueObjectChunkFilter(Object value1, Object value2) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Object checkValue = values.get(ii); - if (Objects.equals(checkValue, value1) || Objects.equals(checkValue, value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return Objects.equals(value, value1) || Objects.equals(value, value2); } } - private static class InverseTwoValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class InverseTwoValueObjectChunkFilter extends ObjectChunkFilter { private final Object value1; private final Object value2; @@ -110,19 +88,12 @@ private InverseTwoValueObjectChunkFilter(Object value1, Object value2) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Object checkValue = values.get(ii); - if (!(Objects.equals(checkValue, value1) || Objects.equals(checkValue, value2))) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return !(Objects.equals(value, value1) || Objects.equals(value, value2)); } } - private static class ThreeValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class ThreeValueObjectChunkFilter extends ObjectChunkFilter { private final Object value1; private final Object value2; private final Object value3; @@ -134,19 +105,12 @@ private ThreeValueObjectChunkFilter(Object value1, Object value2, Object value3) } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Object checkValue = values.get(ii); - if (Objects.equals(checkValue, value1) || Objects.equals(checkValue, value2) - || Objects.equals(checkValue, value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return Objects.equals(value, value1) || Objects.equals(value, value2) || Objects.equals(value, value3); } } - private static class InverseThreeValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + + private final static class InverseThreeValueObjectChunkFilter extends ObjectChunkFilter { private final Object value1; private final Object value2; private final Object value3; @@ -158,20 +122,12 @@ private InverseThreeValueObjectChunkFilter(Object value1, Object value2, Object } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Object checkValue = values.get(ii); - if (!(Objects.equals(checkValue, value1) || Objects.equals(checkValue, value2) - || Objects.equals(checkValue, value3))) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return !(Objects.equals(value, value1) || Objects.equals(value, value2) || Objects.equals(value, value3)); } } - private static class MultiValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class MultiValueObjectChunkFilter extends ObjectChunkFilter { private final HashSet values; private MultiValueObjectChunkFilter(Object... values) { @@ -179,18 +135,12 @@ private MultiValueObjectChunkFilter(Object... values) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (this.values.contains(values.get(ii))) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return this.values.contains(value); } } - private static class InverseMultiValueObjectChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class InverseMultiValueObjectChunkFilter extends ObjectChunkFilter { private final HashSet values; private InverseMultiValueObjectChunkFilter(Object... values) { @@ -198,14 +148,8 @@ private InverseMultiValueObjectChunkFilter(Object... values) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (!this.values.contains(values.get(ii))) { - results.add(keys.get(ii)); - } - } + public boolean matches(Object value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkFilter.java new file mode 100644 index 00000000000..d2aac89d041 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkFilter.java @@ -0,0 +1,67 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +// ****** AUTO-GENERATED CLASS - DO NOT EDIT MANUALLY +// ****** Edit CharChunkFilter and run "./gradlew replicateChunkFilters" to regenerate +// +// @formatter:off +package io.deephaven.engine.table.impl.chunkfilter; + +import io.deephaven.chunk.*; +import io.deephaven.chunk.attributes.Values; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; + +public abstract class ShortChunkFilter implements ChunkFilter { + public abstract boolean matches(short value); + + @Override + public final void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { + final ShortChunk shortChunk = values.asShortChunk(); + final int len = shortChunk.size(); + + results.setSize(0); + for (int ii = 0; ii < len; ++ii) { + if (matches(shortChunk.get(ii))) { + results.add(keys.get(ii)); + } + } + } + + @Override + public final int filter(final Chunk values, final WritableBooleanChunk results) { + final ShortChunk shortChunk = values.asShortChunk(); + final int len = values.size(); + int count = 0; + for (int ii = 0; ii < len; ++ii) { + final boolean newResult = matches(shortChunk.get(ii)); + results.set(ii, newResult); + // count every true value + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public final int filterAnd(final Chunk values, final WritableBooleanChunk results) { + final ShortChunk shortChunk = values.asShortChunk(); + final int len = values.size(); + int count = 0; + // Count the values that changed from true to false + for (int ii = 0; ii < len; ++ii) { + final boolean result = results.get(ii); + if (!result) { + // already false, no need to compute or increment the count + continue; + } + boolean newResult = matches(shortChunk.get(ii)); + results.set(ii, newResult); + // increment the count if the new result is TRUE + count += newResult ? 1 : 0; + } + return count; + } +} + diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkMatchFilterFactory.java index 56be76311c5..d62966e309e 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortChunkMatchFilterFactory.java @@ -7,23 +7,20 @@ // @formatter:off package io.deephaven.engine.table.impl.chunkfilter; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import gnu.trove.set.hash.TShortHashSet; /** * Creates chunk filters for short values. - * + *

* The strategy is that for one, two, or three values we have specialized classes that will do the appropriate simple * equality check. - * + *

* For more values, we use a trove set and check contains for each value in the chunk. */ public class ShortChunkMatchFilterFactory { private ShortChunkMatchFilterFactory() {} // static use only - public static ChunkFilter.ShortChunkFilter makeFilter(boolean invertMatch, short... values) { + public static ShortChunkFilter makeFilter(boolean invertMatch, short... values) { if (invertMatch) { if (values.length == 1) { return new InverseSingleValueShortChunkFilter(values[0]); @@ -49,7 +46,7 @@ public static ChunkFilter.ShortChunkFilter makeFilter(boolean invertMatch, short } } - private static class SingleValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class SingleValueShortChunkFilter extends ShortChunkFilter { private final short value; private SingleValueShortChunkFilter(short value) { @@ -57,18 +54,12 @@ private SingleValueShortChunkFilter(short value) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) == value) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return value == this.value; } } - private static class InverseSingleValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class InverseSingleValueShortChunkFilter extends ShortChunkFilter { private final short value; private InverseSingleValueShortChunkFilter(short value) { @@ -76,18 +67,12 @@ private InverseSingleValueShortChunkFilter(short value) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - if (values.get(ii) != value) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return value != this.value; } } - private static class TwoValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class TwoValueShortChunkFilter extends ShortChunkFilter { private final short value1; private final short value2; @@ -97,19 +82,12 @@ private TwoValueShortChunkFilter(short value1, short value2) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return value == value1 || value == value2; } } - private static class InverseTwoValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class InverseTwoValueShortChunkFilter extends ShortChunkFilter { private final short value1; private final short value2; @@ -119,19 +97,12 @@ private InverseTwoValueShortChunkFilter(short value1, short value2) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2)) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return value != value1 && value != value2; } } - private static class ThreeValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class ThreeValueShortChunkFilter extends ShortChunkFilter { private final short value1; private final short value2; private final short value3; @@ -143,19 +114,12 @@ private ThreeValueShortChunkFilter(short value1, short value2, short value3) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short checkValue = values.get(ii); - if (checkValue == value1 || checkValue == value2 || checkValue == value3) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return value == value1 || value == value2 || value == value3; } } - private static class InverseThreeValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class InverseThreeValueShortChunkFilter extends ShortChunkFilter { private final short value1; private final short value2; private final short value3; @@ -167,19 +131,12 @@ private InverseThreeValueShortChunkFilter(short value1, short value2, short valu } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short checkValue = values.get(ii); - if (!(checkValue == value1 || checkValue == value2 || checkValue == value3)) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return value != value1 && value != value2 && value != value3; } } - private static class MultiValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class MultiValueShortChunkFilter extends ShortChunkFilter { private final TShortHashSet values; private MultiValueShortChunkFilter(short... values) { @@ -187,19 +144,12 @@ private MultiValueShortChunkFilter(short... values) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short checkValue = values.get(ii); - if (this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return this.values.contains(value); } } - private static class InverseMultiValueShortChunkFilter implements ChunkFilter.ShortChunkFilter { + private final static class InverseMultiValueShortChunkFilter extends ShortChunkFilter { private final TShortHashSet values; private InverseMultiValueShortChunkFilter(short... values) { @@ -207,15 +157,8 @@ private InverseMultiValueShortChunkFilter(short... values) { } @Override - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short checkValue = values.get(ii); - if (!this.values.contains(checkValue)) { - results.add(keys.get(ii)); - } - } + public boolean matches(short value) { + return !this.values.contains(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortRangeComparator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortRangeComparator.java index cbddc507b4b..2b994a0f335 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortRangeComparator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/ShortRangeComparator.java @@ -8,14 +8,11 @@ package io.deephaven.engine.table.impl.chunkfilter; import io.deephaven.util.compare.ShortComparisons; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; public class ShortRangeComparator { private ShortRangeComparator() {} // static use only - private abstract static class ShortShortFilter implements ChunkFilter.ShortChunkFilter { + private abstract static class ShortShortFilter extends ShortChunkFilter { final short lower; final short upper; @@ -23,80 +20,53 @@ private abstract static class ShortShortFilter implements ChunkFilter.ShortChunk this.lower = lower; this.upper = upper; } - - abstract public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results); } - static class ShortShortInclusiveInclusiveFilter extends ShortShortFilter { + final static class ShortShortInclusiveInclusiveFilter extends ShortShortFilter { private ShortShortInclusiveInclusiveFilter(short lower, short upper) { super(lower, upper); } - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short value = values.get(ii); - if (ShortComparisons.geq(value, lower) && ShortComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(short value) { + return ShortComparisons.geq(value, lower) && ShortComparisons.leq(value, upper); } } - static class ShortShortInclusiveExclusiveFilter extends ShortShortFilter { + final static class ShortShortInclusiveExclusiveFilter extends ShortShortFilter { private ShortShortInclusiveExclusiveFilter(short lower, short upper) { super(lower, upper); } - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short value = values.get(ii); - if (ShortComparisons.geq(value, lower) && ShortComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(short value) { + return ShortComparisons.geq(value, lower) && ShortComparisons.lt(value, upper); } } - static class ShortShortExclusiveInclusiveFilter extends ShortShortFilter { + final static class ShortShortExclusiveInclusiveFilter extends ShortShortFilter { private ShortShortExclusiveInclusiveFilter(short lower, short upper) { super(lower, upper); } - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short value = values.get(ii); - if (ShortComparisons.gt(value, lower) && ShortComparisons.leq(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(short value) { + return ShortComparisons.gt(value, lower) && ShortComparisons.leq(value, upper); } } - static class ShortShortExclusiveExclusiveFilter extends ShortShortFilter { + final static class ShortShortExclusiveExclusiveFilter extends ShortShortFilter { private ShortShortExclusiveExclusiveFilter(short lower, short upper) { super(lower, upper); } - public void filter(ShortChunk values, LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final short value = values.get(ii); - if (ShortComparisons.gt(value, lower) && ShortComparisons.lt(value, upper)) { - results.add(keys.get(ii)); - } - } + @Override + public boolean matches(short value) { + return ShortComparisons.gt(value, lower) && ShortComparisons.lt(value, upper); } } - public static ChunkFilter.ShortChunkFilter makeShortFilter(short lower, short upper, boolean lowerInclusive, + public static ShortChunkFilter makeShortFilter(short lower, short upper, boolean lowerInclusive, boolean upperInclusive) { if (lowerInclusive) { if (upperInclusive) { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/StringChunkMatchFilterFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/StringChunkMatchFilterFactory.java index b99a5b5d61d..d07d0b5a380 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/StringChunkMatchFilterFactory.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/StringChunkMatchFilterFactory.java @@ -6,9 +6,6 @@ import io.deephaven.hash.KeyedObjectHashSet; import io.deephaven.hash.KeyedObjectKey; import io.deephaven.base.string.cache.CharSequenceUtils; -import io.deephaven.chunk.*; -import io.deephaven.chunk.attributes.Values; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; class StringChunkMatchFilterFactory { private static final class CIStringKey implements KeyedObjectKey { @@ -32,7 +29,8 @@ public boolean equalKey(String s, String s2) { private StringChunkMatchFilterFactory() {} // static use only - static ChunkFilter.ObjectChunkFilter makeCaseInsensitiveFilter(boolean invert, Object... values) { + @SuppressWarnings("rawtypes") + static ObjectChunkFilter makeCaseInsensitiveFilter(boolean invert, Object... values) { if (invert) { if (values.length == 1) { return new InverseSingleValueStringChunkFilter((String) values[0]); @@ -59,7 +57,7 @@ static ChunkFilter.ObjectChunkFilter makeCaseInsensitiveFilter(boolean invert, O } } - private static class SingleValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private final static class SingleValueStringChunkFilter extends ObjectChunkFilter { private final String value; private SingleValueStringChunkFilter(String value) { @@ -67,20 +65,12 @@ private SingleValueStringChunkFilter(String value) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (value.equalsIgnoreCase(checkString)) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return this.value.equalsIgnoreCase(value); } } - private static class InverseSingleValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class InverseSingleValueStringChunkFilter extends ObjectChunkFilter { private final String value; private InverseSingleValueStringChunkFilter(String value) { @@ -88,20 +78,12 @@ private InverseSingleValueStringChunkFilter(String value) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (!value.equalsIgnoreCase(checkString)) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return !this.value.equalsIgnoreCase(value); } } - private static class TwoValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class TwoValueStringChunkFilter extends ObjectChunkFilter { private final String value1; private final String value2; @@ -111,20 +93,12 @@ private TwoValueStringChunkFilter(String value1, String value2) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (value1.equalsIgnoreCase(checkString) || value2.equalsIgnoreCase(checkString)) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return value1.equalsIgnoreCase(value) || value2.equalsIgnoreCase(value); } } - private static class InverseTwoValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class InverseTwoValueStringChunkFilter extends ObjectChunkFilter { private final String value1; private final String value2; @@ -134,20 +108,12 @@ private InverseTwoValueStringChunkFilter(String value1, String value2) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (!(value1.equalsIgnoreCase(checkString) || value2.equalsIgnoreCase(checkString))) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return !value1.equalsIgnoreCase(value) && !value2.equalsIgnoreCase(value); } } - private static class ThreeValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class ThreeValueStringChunkFilter extends ObjectChunkFilter { private final String value1; private final String value2; private final String value3; @@ -159,21 +125,12 @@ private ThreeValueStringChunkFilter(String value1, String value2, String value3) } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (value1.equalsIgnoreCase(checkString) || value2.equalsIgnoreCase(checkString) - || value3.equalsIgnoreCase(checkString)) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return value1.equalsIgnoreCase(value) || value2.equalsIgnoreCase(value) || value3.equalsIgnoreCase(value); } } - private static class InverseThreeValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class InverseThreeValueStringChunkFilter extends ObjectChunkFilter { private final String value1; private final String value2; private final String value3; @@ -185,21 +142,13 @@ private InverseThreeValueStringChunkFilter(String value1, String value2, String } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (!(value1.equalsIgnoreCase(checkString) || value2.equalsIgnoreCase(checkString) - || value3.equalsIgnoreCase(checkString))) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return !value1.equalsIgnoreCase(value) && !value2.equalsIgnoreCase(value) + && !value3.equalsIgnoreCase(value); } } - private static class MultiValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class MultiValueStringChunkFilter extends ObjectChunkFilter { private final KeyedObjectHashSet values; private MultiValueStringChunkFilter(Object... values) { @@ -210,20 +159,12 @@ private MultiValueStringChunkFilter(Object... values) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (this.values.containsKey(checkString)) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return this.values.containsKey(value); } } - private static class InverseMultiValueStringChunkFilter implements ChunkFilter.ObjectChunkFilter { + private static class InverseMultiValueStringChunkFilter extends ObjectChunkFilter { private final KeyedObjectHashSet values; private InverseMultiValueStringChunkFilter(Object... values) { @@ -234,16 +175,8 @@ private InverseMultiValueStringChunkFilter(Object... values) { } @Override - public void filter(ObjectChunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk stringChunk = values.asTypedObjectChunk(); - results.setSize(0); - for (int ii = 0; ii < stringChunk.size(); ++ii) { - final String checkString = stringChunk.get(ii); - if (!this.values.containsKey(checkString)) { - results.add(keys.get(ii)); - } - } + public boolean matches(String value) { + return !this.values.containsKey(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/SupportedRangeJoinAggregations.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/SupportedRangeJoinAggregations.java index 072db53b3c1..5a21b8d570d 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/SupportedRangeJoinAggregations.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/SupportedRangeJoinAggregations.java @@ -67,6 +67,11 @@ public void visit(@NotNull final Count count) { hasUnsupportedAggs = true; } + @Override + public void visit(@NotNull final CountWhere countWhere) { + hasUnsupportedAggs = true; + } + @Override public void visit(@NotNull final FirstRowKey firstRowKey) { hasUnsupportedAggs = true; diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/ValidFloatingPointFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/ValidFloatingPointFilter.java index 686e4ab7130..767b9a829d0 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/ValidFloatingPointFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/rangejoin/ValidFloatingPointFilter.java @@ -5,19 +5,15 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.Strings; -import io.deephaven.chunk.DoubleChunk; -import io.deephaven.chunk.FloatChunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.DoubleChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.FloatChunkFilter; import io.deephaven.engine.table.impl.select.WhereFilter; import io.deephaven.engine.table.impl.select.WhereFilterImpl; import org.jetbrains.annotations.NotNull; @@ -86,47 +82,27 @@ public WritableRowSet filter( return ChunkFilter.applyChunkFilter(selection, columnSource, usePrev, chunkFilter); } - private static final class DoubleFilter implements ChunkFilter.DoubleChunkFilter { + private static final class DoubleFilter extends DoubleChunkFilter { private static final DoubleFilter INSTANCE = new DoubleFilter(); private DoubleFilter() {} @Override - public void filter( - @NotNull final DoubleChunk values, - @NotNull final LongChunk rowKeys, - @NotNull final WritableLongChunk acceptedRowKeys) { - final int size = values.size(); - acceptedRowKeys.setSize(0); - for (int vi = 0; vi < size; ++vi) { - final double value = values.get(vi); - if (!Double.isNaN(value) && value != NULL_DOUBLE) { - acceptedRowKeys.add(rowKeys.get(vi)); - } - } + public boolean matches(final double value) { + return !Double.isNaN(value) && value != NULL_DOUBLE; } } - private static final class FloatFilter implements ChunkFilter.FloatChunkFilter { + private static final class FloatFilter extends FloatChunkFilter { private static final FloatFilter INSTANCE = new FloatFilter(); private FloatFilter() {} @Override - public void filter( - @NotNull final FloatChunk values, - @NotNull final LongChunk rowKeys, - @NotNull final WritableLongChunk acceptedRowKeys) { - final int size = values.size(); - acceptedRowKeys.setSize(0); - for (int vi = 0; vi < size; ++vi) { - final float value = values.get(vi); - if (!Float.isNaN(value) && value != NULL_FLOAT) { - acceptedRowKeys.add(rowKeys.get(vi)); - } - } + public boolean matches(final float value) { + return !Float.isNaN(value) && value != NULL_FLOAT; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractConditionFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractConditionFilter.java index 9b9602843f5..83f60cadda2 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractConditionFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/AbstractConditionFilter.java @@ -4,9 +4,14 @@ package io.deephaven.engine.table.impl.select; import io.deephaven.base.Pair; +import io.deephaven.chunk.Chunk; +import io.deephaven.chunk.LongChunk; +import io.deephaven.chunk.WritableBooleanChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.context.QueryScopeParam; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; +import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.impl.MatchPair; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; @@ -168,7 +173,7 @@ public void validateSafeForRefresh(BaseTable sourceTable) { return; } if (sourceTable.isRefreshing() && !AbstractFormulaColumn.ALLOW_UNSAFE_REFRESHING_FORMULAS) { - // note that constant offset array accesss does not use i/ii or end up in usedColumnArrays + // note that constant offset array access does not use i/ii or end up in usedColumnArrays boolean isUnsafe = (usesI || usesII) && !sourceTable.isAppendOnly() && !sourceTable.isBlink(); isUnsafe |= usesK && !sourceTable.isAddOnly() && !sourceTable.isBlink(); isUnsafe |= !usedColumnArrays.isEmpty() && !sourceTable.isBlink(); @@ -262,7 +267,19 @@ public WritableRowSet filter( return filter.filter(selection, fullSet, table, usePrev, formula, params); } - protected abstract Filter getFilter(Table table, RowSet fullSet) + /** + * Retrieve the current {@link Filter filter} for this condition filter or create a new one initialized to the + * provided table and row set. With a {@link ConditionFilter.FilterKernel.Context context} from + * {@link Filter#getContext(int)}, this filter can be used for directly filtering chunked data. + * + * @param table the table to filter, or a table with a compatible schema + * @param fullSet the full set of rows currently in the table, used to populate the virtual row variables such as + * {@code i}, {@code ii}, and {@code k} + * + * @return the initialized filter + */ + @NotNull + public abstract Filter getFilter(Table table, RowSet fullSet) throws InstantiationException, IllegalAccessException, NoSuchMethodException, InvocationTargetException; /** @@ -311,7 +328,11 @@ public boolean hasConstantArrayAccess() { return getFormulaShiftColPair() != null; } - @Override + /** + * Returns true if this filters uses row virtual offset columns of {@code i}, {@code ii} or {@code k}. + *

+ * This filter must already be initialized before calling this method. + */ public boolean hasVirtualRowVariables() { return usesI || usesII || usesK; } @@ -339,6 +360,48 @@ WritableRowSet filter( boolean usePrev, String formula, QueryScopeParam... params); + + /** + * Create a new context for this filter, must be closed after use. + */ + ConditionFilter.FilterKernel.Context getContext(int chunkSize); + + /** + * Filter a chunk of values and copy the matching row keys to the returned chunk. + */ + LongChunk filter( + ConditionFilter.FilterKernel.Context context, + LongChunk inputKeys, + Chunk[] valueChunks); + + /** + * Filter a chunk of values, setting parallel values in {@code results} to the output of the filter. + * + * @return the number of values are {@code true} in {@code results} after the filter is applied. + */ + int filter( + ConditionFilter.FilterKernel.Context context, + Chunk[] valueChunks, + int chunkSize, + WritableBooleanChunk results); + + /** + * Filter a chunk of values, setting parallel values in {@code results} to {@code false} when the filter result + * is {@code false}. The filter will not be evaluated for values that are currently {@code false} in the results + * chunk. + *

+ * To use this method effectively, the results chunk should be initialized by a call to + * {@link #filter(ConditionFilter.FilterKernel.Context, Chunk[], int, WritableBooleanChunk)} or by setting all + * values to {@code true} before the first call. Successive calls will have the effect of AND'ing this filter + * results with existing results. + * + * @return the number of values are {@code true} in {@code results} after the filter is applied. + */ + int filterAnd( + ConditionFilter.FilterKernel.Context context, + Chunk[] valueChunks, + int chunkSize, + WritableBooleanChunk results); } static String truncateLongFormula(String formula) { 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 513dafc7d36..2e4edee65f3 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 @@ -23,7 +23,7 @@ * A filter that determines if a column value is between an upper and lower bound (which each may either be inclusive or * exclusive). */ -public abstract class AbstractRangeFilter extends WhereFilterImpl { +public abstract class AbstractRangeFilter extends WhereFilterImpl implements ExposesChunkFilter { private static final Pattern decimalPattern = Pattern.compile("(-)?\\d+(?:\\.((\\d+)0*)?)?"); protected final String columnName; @@ -49,6 +49,11 @@ public abstract class AbstractRangeFilter extends WhereFilterImpl { this.lowerInclusive = lowerInclusive; } + @Override + public Optional chunkFilter() { + return Optional.of(chunkFilter); + } + public static WhereFilter makeBigDecimalRange(String columnName, String val) { final int precision = findPrecision(val); final BigDecimal parsed = new BigDecimal(val); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComparableRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComparableRangeFilter.java index 3050b8d13a3..84a9f85e007 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComparableRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ComparableRangeFilter.java @@ -7,11 +7,9 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.ObjectChunkFilter; import io.deephaven.util.compare.ObjectComparisons; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.util.annotations.TestUseOnly; @@ -91,7 +89,8 @@ public String toString() { (upperInclusive ? "]" : ")") + ")"; } - private static class InclusiveInclusiveComparableChunkFilter implements ChunkFilter { + private final static class InclusiveInclusiveComparableChunkFilter + extends ObjectChunkFilter> { private final Comparable lower; private final Comparable upper; @@ -101,29 +100,13 @@ private InclusiveInclusiveComparableChunkFilter(Comparable lower, Comparable< } @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (meetsLowerBound(value) && meetsUpperBound(value)) { - results.add(keys.get(ii)); - } - } - } - - boolean meetsLowerBound(Comparable value) { - return ObjectComparisons.compare(lower, value) <= 0; - } - - boolean meetsUpperBound(Comparable value) { - return ObjectComparisons.compare(upper, value) >= 0; + public boolean matches(Comparable value) { + return ObjectComparisons.compare(lower, value) <= 0 && ObjectComparisons.compare(upper, value) >= 0; } } - private static class InclusiveExclusiveComparableChunkFilter implements ChunkFilter { + private final static class InclusiveExclusiveComparableChunkFilter + extends ObjectChunkFilter> { private final Comparable lower; private final Comparable upper; @@ -132,31 +115,14 @@ private InclusiveExclusiveComparableChunkFilter(Comparable lower, Comparable< this.upper = upper; } - @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (meetsLowerBound(value) && meetsUpperBound(value)) { - results.add(keys.get(ii)); - } - } - } - - boolean meetsLowerBound(Comparable value) { - return ObjectComparisons.compare(lower, value) <= 0; - } - - boolean meetsUpperBound(Comparable value) { - return ObjectComparisons.compare(upper, value) > 0; + public boolean matches(Comparable value) { + return ObjectComparisons.compare(lower, value) <= 0 && ObjectComparisons.compare(upper, value) > 0; } } - private static class ExclusiveInclusiveComparableChunkFilter implements ChunkFilter { + private final static class ExclusiveInclusiveComparableChunkFilter + extends ObjectChunkFilter> { private final Comparable lower; private final Comparable upper; @@ -165,31 +131,14 @@ private ExclusiveInclusiveComparableChunkFilter(Comparable lower, Comparable< this.upper = upper; } - @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (meetsLowerBound(value) && meetsUpperBound(value)) { - results.add(keys.get(ii)); - } - } - } - - boolean meetsLowerBound(Comparable value) { - return ObjectComparisons.compare(lower, value) < 0; - } - - boolean meetsUpperBound(Comparable value) { - return ObjectComparisons.compare(upper, value) >= 0; + public boolean matches(Comparable value) { + return ObjectComparisons.compare(lower, value) < 0 && ObjectComparisons.compare(upper, value) >= 0; } } - private static class ExclusiveExclusiveComparableChunkFilter implements ChunkFilter { + private final static class ExclusiveExclusiveComparableChunkFilter + extends ObjectChunkFilter> { private final Comparable lower; private final Comparable upper; @@ -198,27 +147,9 @@ private ExclusiveExclusiveComparableChunkFilter(Comparable lower, Comparable< this.upper = upper; } - @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (meetsLowerBound(value) && meetsUpperBound(value)) { - results.add(keys.get(ii)); - } - } - } - - boolean meetsLowerBound(Comparable value) { - return ObjectComparisons.compare(lower, value) < 0; - } - - boolean meetsUpperBound(Comparable value) { - return ObjectComparisons.compare(upper, value) > 0; + public boolean matches(Comparable value) { + return ObjectComparisons.compare(lower, value) < 0 && ObjectComparisons.compare(upper, value) > 0; } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ConditionFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ConditionFilter.java index 51351cfb516..1868317c79a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ConditionFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ConditionFilter.java @@ -5,6 +5,7 @@ import io.deephaven.base.Pair; import io.deephaven.chunk.attributes.Any; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.context.QueryCompilerImpl; import io.deephaven.engine.context.QueryCompilerRequest; @@ -47,7 +48,6 @@ * A condition filter evaluates a formula against a table. */ public class ConditionFilter extends AbstractConditionFilter { - public static final int CHUNK_SIZE = 4096; private Future> filterKernelClassFuture = null; private List>> usedInputs; // that is columns and special variables @@ -113,8 +113,19 @@ public void close() { CONTEXT getContext(int maxChunkSize); - LongChunk filter(CONTEXT context, LongChunk indices, - Chunk... inputChunks); + LongChunk filter(CONTEXT context, LongChunk indices, Chunk... inputChunks); + + int filter( + CONTEXT context, + Chunk[] inputChunks, + int chunkSize, + WritableBooleanChunk results); + + int filterAnd( + CONTEXT context, + Chunk[] inputChunks, + int chunkSize, + WritableBooleanChunk results); } @@ -375,6 +386,37 @@ public WritableRowSet filter(final RowSet selection, final RowSet fullSet, final return resultBuilder.build(); } } + + @Override + public FilterKernel.Context getContext(int chunkSize) { + return filterKernel.getContext(chunkSize); + } + + @Override + public LongChunk filter( + final FilterKernel.Context context, + final LongChunk inputKeys, + final Chunk[] valueChunks) { + return filterKernel.filter(context, inputKeys, valueChunks); + } + + @Override + public int filter( + final FilterKernel.Context context, + final Chunk[] valueChunks, + final int chunkSize, + final WritableBooleanChunk results) { + return filterKernel.filter(context, valueChunks, chunkSize, results); + } + + @Override + public int filterAnd( + final FilterKernel.Context context, + final Chunk[] valueChunks, + final int chunkSize, + final WritableBooleanChunk results) { + return filterKernel.filterAnd(context, valueChunks, chunkSize, results); + } } private static String toTitleCase(String input) { @@ -550,22 +592,94 @@ private StringBuilder getClassBody( "@Override\n" + "public LongChunk filter(Context __context, LongChunk __indices, Chunk... __inputChunks) {\n"); indenter.increaseLevel(); - for (int i = 0; i < usedInputs.size(); i++) { - final Class columnType = usedInputs.get(i).second; - final String chunkType; - if (columnType.isPrimitive() && columnType != boolean.class) { - chunkType = toTitleCase(columnType.getSimpleName()) + "Chunk"; - } else { - // TODO: Reinterpret Boolean and Instant to byte and long - chunkType = "ObjectChunk"; - } - classBody.append(indenter).append("final ").append(chunkType).append(" __columnChunk").append(i) - .append(" = __inputChunks[").append(i).append("].as").append(chunkType).append("();\n"); - } + insertChunks(classBody, indenter); indenter.indent(classBody, "final int __size = __indices.size();\n" + "__context.resultChunk.setSize(0);\n" + "for (int __my_i__ = 0; __my_i__ < __size; __my_i__++) {\n"); indenter.increaseLevel(); + insertChunkValues(classBody, indenter); + classBody.append("" + + " if (").append(result.getConvertedExpression()).append(") {\n" + + " __context.resultChunk.add(__indices.get(__my_i__));\n" + + " }\n" + + " }\n" + + " return __context.resultChunk;\n" + + " }\n"); + indenter.decreaseLevel(); + indenter.decreaseLevel(); + + ////////////////////////////////// + + indenter.indent(classBody, "\n" + + "@Override\n" + + "public int filter(final Context __context, final Chunk[] __inputChunks, final int __chunkSize, final WritableBooleanChunk __results) {\n"); + indenter.increaseLevel(); + insertChunks(classBody, indenter); + + indenter.indent(classBody, "" + + "__results.setSize(__chunkSize);\n" + + "int __count = 0;\n" + + "for (int __my_i__ = 0; __my_i__ < __chunkSize; __my_i__++) {"); + indenter.increaseLevel(); + insertChunkValues(classBody, indenter); + indenter.indent(classBody, "" + + "final boolean __newResult = " + result.getConvertedExpression() + ";\n" + + "__results.set(__my_i__, __newResult);\n" + + "// count every true value\n" + + "__count += __newResult ? 1 : 0;\n"); + indenter.decreaseLevel(); + indenter.indent(classBody, "" + + "}\n" + + "return __count;"); + indenter.decreaseLevel(); + indenter.indent(classBody, "" + + "}\n"); + + ////////////////////////////////// + + indenter.indent(classBody, "\n" + + "@Override\n" + + "public int filterAnd(final Context __context, final Chunk[] __inputChunks, final int __chunkSize, final WritableBooleanChunk __results) {\n"); + indenter.increaseLevel(); + insertChunks(classBody, indenter); + + indenter.indent(classBody, "" + + "__results.setSize(__chunkSize);\n" + + "int __count = 0;\n" + + "for (int __my_i__ = 0; __my_i__ < __chunkSize; __my_i__++) {\n"); + indenter.increaseLevel(); + indenter.indent(classBody, "" + + "final boolean __result = __results.get(__my_i__);\n" + + "if (!__result) {\n" + + " // already false, no need to compute or increment the count\n" + + " continue;\n" + + "}"); + insertChunkValues(classBody, indenter); + indenter.indent(classBody, "" + + "final boolean __newResult = " + result.getConvertedExpression() + ";\n" + + "__results.set(__my_i__, __newResult);\n" + + "__results.set(__my_i__, __newResult);\n" + + "// increment the count if the new result is TRUE\n" + + "__count += __newResult ? 1 : 0;\n"); + + indenter.decreaseLevel(); + indenter.indent(classBody, "" + + "}\n" + + "return __count;"); + indenter.decreaseLevel(); + indenter.indent(classBody, "" + + "}\n\n"); + + ////////////////////////////////// + + indenter.decreaseLevel(); + indenter.indent(classBody, "" + + "}\n\n"); + + return classBody; + } + + private void insertChunkValues(StringBuilder classBody, Indenter indenter) { for (int i = 0; i < usedInputs.size(); i++) { final Pair> usedInput = usedInputs.get(i); final Class columnType = usedInput.second; @@ -574,19 +688,26 @@ private StringBuilder getClassBody( .append(" = (").append(canonicalName).append(")__columnChunk").append(i) .append(".get(__my_i__);\n"); } - classBody.append( - " if (").append(result.getConvertedExpression()).append(") {\n" + - " __context.resultChunk.add(__indices.get(__my_i__));\n" + - " }\n" + - " }\n" + - " return __context.resultChunk;\n" + - " }\n" + - "}"); - return classBody; + } + + private void insertChunks(StringBuilder classBody, Indenter indenter) { + for (int i = 0; i < usedInputs.size(); i++) { + final Class columnType = usedInputs.get(i).second; + final String chunkType; + if (columnType.isPrimitive() && columnType != boolean.class) { + chunkType = toTitleCase(columnType.getSimpleName()) + "Chunk"; + } else { + // TODO: Reinterpret Boolean and Instant to byte and long + chunkType = "ObjectChunk"; + } + classBody.append(indenter).append("final ").append(chunkType).append(" __columnChunk").append(i) + .append(" = __inputChunks[").append(i).append("].as").append(chunkType).append("();\n"); + } } @Override - protected Filter getFilter(Table table, RowSet fullSet) + @NotNull + public Filter getFilter(Table table, RowSet fullSet) throws InstantiationException, IllegalAccessException, NoSuchMethodException, InvocationTargetException { if (filter == null) { try { diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java new file mode 100644 index 00000000000..5c46f82ec47 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/ExposesChunkFilter.java @@ -0,0 +1,17 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.select; + +import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; + +import java.util.Optional; + +public interface ExposesChunkFilter { + /** + * Retrieve the underlying chunk filter for this filter if available. + * + * @return If available, returns the underlying chunk filter. Otherwise returns {@link Optional#empty()}. + */ + Optional chunkFilter(); +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/InstantRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/InstantRangeFilter.java index 7d45254db75..64659acca44 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/InstantRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/InstantRangeFilter.java @@ -90,19 +90,43 @@ WritableRowSet binarySearch( } private class InstantLongChunkFilterAdapter implements ChunkFilter { + /** + * Convert an Instant chunk to a Long chunk. + */ + private WritableLongChunk convert(final Chunk values) { + final ObjectChunk objectValues = values.asObjectChunk(); + final WritableLongChunk outputChunk = WritableLongChunk.makeWritableChunk(objectValues.size()); + for (int ii = 0; ii < values.size(); ++ii) { + final Instant instant = objectValues.get(ii); + outputChunk.set(ii, DateTimeUtils.epochNanos(instant)); + } + outputChunk.setSize(values.size()); + return outputChunk; + } + @Override public void filter(Chunk values, LongChunk keys, WritableLongChunk results) { - try (final WritableLongChunk writableLongChunk = - WritableLongChunk.makeWritableChunk(values.size())) { - - final ObjectChunk objectValues = values.asObjectChunk(); - for (int ii = 0; ii < values.size(); ++ii) { - final Instant instant = objectValues.get(ii); - writableLongChunk.set(ii, DateTimeUtils.epochNanos(instant)); - } - writableLongChunk.setSize(values.size()); - longFilter.filter(writableLongChunk, keys, results); + try (final WritableLongChunk convertedChunk = convert(values)) { + longFilter.filter(convertedChunk, keys, results); + } + } + + @Override + public int filter( + final Chunk values, + final WritableBooleanChunk results) { + try (final WritableLongChunk convertedChunk = convert(values)) { + return longFilter.filter(convertedChunk, results); + } + } + + @Override + public int filterAnd( + final Chunk values, + final WritableBooleanChunk results) { + try (final WritableLongChunk convertedChunk = convert(values)) { + return longFilter.filterAnd(convertedChunk, results); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java index 9d7353758b0..1052a8a96fb 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/MatchFilter.java @@ -15,6 +15,8 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.QueryCompilerRequestProcessor; +import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.ChunkMatchFilterFactory; import io.deephaven.engine.table.impl.lang.QueryLanguageFunctionUtils; import io.deephaven.engine.table.impl.preview.DisplayWrapper; import io.deephaven.engine.table.impl.DependencyStreamProvider; @@ -41,7 +43,7 @@ import java.util.function.Consumer; import java.util.stream.Stream; -public class MatchFilter extends WhereFilterImpl implements DependencyStreamProvider { +public class MatchFilter extends WhereFilterImpl implements DependencyStreamProvider, ExposesChunkFilter { private static final long serialVersionUID = 1L; @@ -60,6 +62,7 @@ static MatchFilter ofLiterals( @NotNull private String columnName; + private Class columnType; private Object[] values; private final String[] strValues; private final boolean invertMatch; @@ -222,6 +225,7 @@ public synchronized void init( initialized = true; return; } + columnType = column.getDataType(); final List valueList = new ArrayList<>(); final Map queryScopeVariables = compilationProcessor.getFormulaImports().getQueryScopeVariables(); @@ -303,6 +307,16 @@ public WritableRowSet filterInverse( return columnSource.match(!invertMatch, usePrev, caseInsensitive, dataIndex, selection, values); } + private ChunkFilter chunkFilter; + + @Override + public Optional chunkFilter() { + if (chunkFilter == null) { + chunkFilter = ChunkMatchFilterFactory.getChunkFilter(columnType, caseInsensitive, invertMatch, values); + } + return Optional.of(chunkFilter); + } + @Override public boolean isSimpleFilter() { final WhereFilter failover = getFailoverFilterIfCached(); diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java index 5236862c6d5..64fef6fc230 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/RangeFilter.java @@ -8,6 +8,7 @@ import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.QueryCompilerRequestProcessor; +import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; import io.deephaven.time.DateTimeUtils; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; @@ -25,6 +26,7 @@ import java.time.LocalTime; import java.time.ZonedDateTime; import java.util.List; +import java.util.Optional; /** * A filter for comparable types (including Instant) for {@link Condition} values:
@@ -35,7 +37,7 @@ *
  • GREATER_THAN_OR_EQUAL
  • * */ -public class RangeFilter extends WhereFilterImpl { +public class RangeFilter extends WhereFilterImpl implements ExposesChunkFilter { private String columnName; private String value; @@ -264,6 +266,15 @@ public void init( filter.init(tableDefinition, compilationProcessor); } + @Override + public Optional chunkFilter() { + // The underlying filter may be a ConditionFilter + if (filter instanceof ExposesChunkFilter) { + return ((ExposesChunkFilter) filter).chunkFilter(); + } + return Optional.empty(); + } + private static LongRangeFilter makeInstantRangeFilter(String columnName, Condition condition, long value) { switch (condition) { case LESS_THAN: diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/SingleSidedComparableRangeFilter.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/SingleSidedComparableRangeFilter.java index d9a54a7758b..19c5a76c564 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/SingleSidedComparableRangeFilter.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/SingleSidedComparableRangeFilter.java @@ -7,11 +7,9 @@ import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.ObjectChunkFilter; import io.deephaven.util.compare.ObjectComparisons; import io.deephaven.engine.table.ColumnSource; -import io.deephaven.chunk.*; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; -import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.WritableRowSet; import io.deephaven.engine.rowset.RowSet; import io.deephaven.util.annotations.TestUseOnly; @@ -82,7 +80,7 @@ public String toString() { + (lowerInclusive ? "=" : "") + pivot + ")"; } - private static class GeqComparableChunkFilter implements ChunkFilter { + private static class GeqComparableChunkFilter extends ObjectChunkFilter> { private final Comparable pivot; private GeqComparableChunkFilter(Comparable pivot) { @@ -90,21 +88,12 @@ private GeqComparableChunkFilter(Comparable pivot) { } @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (ObjectComparisons.geq(value, pivot)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Comparable value) { + return ObjectComparisons.geq(value, pivot); } } - private static class LeqComparableChunkFilter implements ChunkFilter { + private static class LeqComparableChunkFilter extends ObjectChunkFilter> { private final Comparable pivot; private LeqComparableChunkFilter(Comparable pivot) { @@ -112,21 +101,12 @@ private LeqComparableChunkFilter(Comparable pivot) { } @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (ObjectComparisons.leq(value, pivot)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Comparable value) { + return ObjectComparisons.leq(value, pivot); } } - private static class GtComparableChunkFilter implements ChunkFilter { + private static class GtComparableChunkFilter extends ObjectChunkFilter> { private final Comparable pivot; private GtComparableChunkFilter(Comparable pivot) { @@ -134,21 +114,12 @@ private GtComparableChunkFilter(Comparable pivot) { } @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (ObjectComparisons.gt(value, pivot)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Comparable value) { + return ObjectComparisons.gt(value, pivot); } } - private static class LtComparableChunkFilter implements ChunkFilter { + private static class LtComparableChunkFilter extends ObjectChunkFilter> { private final Comparable pivot; private LtComparableChunkFilter(Comparable pivot) { @@ -156,17 +127,8 @@ private LtComparableChunkFilter(Comparable pivot) { } @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { - final ObjectChunk, ? extends Values> objectChunk = values.asObjectChunk(); - - results.setSize(0); - for (int ii = 0; ii < values.size(); ++ii) { - final Comparable value = objectChunk.get(ii); - if (ObjectComparisons.lt(value, pivot)) { - results.add(keys.get(ii)); - } - } + public boolean matches(Comparable value) { + return ObjectComparisons.lt(value, pivot); } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterPatternImpl.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterPatternImpl.java index de19278857d..395f8a12279 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterPatternImpl.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/WhereFilterPatternImpl.java @@ -5,19 +5,14 @@ import io.deephaven.api.ColumnName; import io.deephaven.api.filter.FilterPattern; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.ObjectChunk; -import io.deephaven.chunk.WritableLongChunk; -import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.rowset.RowSet; import io.deephaven.engine.rowset.WritableRowSet; -import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.engine.table.ColumnDefinition; import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.TableDefinition; import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter; -import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter.ObjectChunkFilter; +import io.deephaven.engine.table.impl.chunkfilter.ObjectChunkFilter; import org.jetbrains.annotations.NotNull; import java.util.List; @@ -80,9 +75,7 @@ public boolean isSimpleFilter() { } @Override - public void setRecomputeListener(RecomputeListener listener) { - - } + public void setRecomputeListener(RecomputeListener listener) {} @Override public boolean canMemoize() { @@ -166,123 +159,59 @@ private boolean findPatternInverted(CharSequence value) { return value != null && !filterPattern.pattern().matcher(value).find(); } - private class Matches implements ObjectChunkFilter { + private final class Matches extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (matches(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return WhereFilterPatternImpl.this.matches(value); } } - private class MatchesPatternInverted implements ObjectChunkFilter { + private final class MatchesPatternInverted extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (matchesPatternInverted(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return WhereFilterPatternImpl.this.matchesPatternInverted(value); } } - private class Find implements ObjectChunkFilter { + private final class Find extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (find(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return WhereFilterPatternImpl.this.find(value); } } - private class FindPatternInverted implements ObjectChunkFilter { + private final class FindPatternInverted extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (findPatternInverted(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return WhereFilterPatternImpl.this.findPatternInverted(value); } } - private class NotMatches implements ObjectChunkFilter { + private final class NotMatches extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (!matches(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return !WhereFilterPatternImpl.this.matches(value); } } - private class NotMatchesPatternInverted implements ObjectChunkFilter { + private final class NotMatchesPatternInverted extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (!matchesPatternInverted(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return !WhereFilterPatternImpl.this.matchesPatternInverted(value); } } - private class NotFind implements ObjectChunkFilter { + private final class NotFind extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (!find(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return !WhereFilterPatternImpl.this.find(value); } } - private class NotFindPatternInverted implements ObjectChunkFilter { + private final class NotFindPatternInverted extends ObjectChunkFilter { @Override - public void filter( - ObjectChunk values, - LongChunk keys, - WritableLongChunk results) { - results.setSize(0); - for (int ix = 0; ix < values.size(); ++ix) { - if (!findPatternInverted(values.get(ix))) { - results.add(keys.get(ix)); - } - } + public boolean matches(CharSequence value) { + return !WhereFilterPatternImpl.this.findPatternInverted(value); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonChunkedFunction.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonChunkedFunction.java index c8bbe7e8667..2a17f476306 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonChunkedFunction.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonChunkedFunction.java @@ -3,6 +3,8 @@ // package io.deephaven.engine.table.impl.select.python; +import io.deephaven.chunk.WritableBooleanChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.impl.select.ConditionFilter.FilterKernel; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.chunk.Chunk; @@ -59,4 +61,57 @@ public LongChunk filter( } return context.resultChunk; } + + @Override + public int filter( + final Context context, + final Chunk[] inputChunks, + final int chunkSize, + final WritableBooleanChunk results) { + FillContextPython fillContextPython = context.getKernelContext(); + fillContextPython.resolveColumnChunks(inputChunks, chunkSize); + + final boolean[] pyResults = function + .call(boolean[].class, CALL_METHOD, fillContextPython.getChunkedArgTypes(), + fillContextPython.getChunkedArgs()); + if (chunkSize > pyResults.length) { + throw new IllegalStateException( + "FilterKernelPythonChunkedFunction returned results are not the proper size"); + } + // Count the number of true values + int count = 0; + for (int i = 0; i < chunkSize; ++i) { + boolean newResult = pyResults[i]; + results.set(i, newResult); + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public int filterAnd( + final Context context, + final Chunk[] inputChunks, + final int chunkSize, + final WritableBooleanChunk results) { + FillContextPython fillContextPython = context.getKernelContext(); + fillContextPython.resolveColumnChunks(inputChunks, chunkSize); + + final boolean[] pyResults = function + .call(boolean[].class, CALL_METHOD, fillContextPython.getChunkedArgTypes(), + fillContextPython.getChunkedArgs()); + if (chunkSize > pyResults.length) { + throw new IllegalStateException( + "FilterKernelPythonChunkedFunction returned results are not the proper size"); + } + // Count values that changed from true to false + int count = 0; + for (int i = 0; i < chunkSize; ++i) { + boolean result = results.get(i); + boolean newResult = result & pyResults[i]; + results.set(i, newResult); + count += result == newResult ? 0 : 1; + } + return count; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonSingularFunction.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonSingularFunction.java index f7a42c84acc..fc88c77ee67 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonSingularFunction.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/select/python/FilterKernelPythonSingularFunction.java @@ -3,6 +3,8 @@ // package io.deephaven.engine.table.impl.select.python; +import io.deephaven.chunk.WritableBooleanChunk; +import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.table.impl.select.ConditionFilter.FilterKernel; import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys; import io.deephaven.chunk.Chunk; @@ -48,4 +50,47 @@ public LongChunk filter( } return context.resultChunk; } + + @Override + public int filter( + final Context context, + final Chunk[] inputChunks, + final int chunkSize, + final WritableBooleanChunk results) { + final Class[] paramTypes = ArgumentsSingular.buildParamTypes(inputChunks); + context.resultChunk.setSize(0); + // Count the number of true values + int count = 0; + for (int i = 0; i < chunkSize; ++i) { + final Object[] params = ArgumentsSingular.buildArguments(inputChunks, i); + boolean newResult = function.call(boolean.class, CALL_METHOD, paramTypes, params); + results.set(i, newResult); + count += newResult ? 1 : 0; + } + return count; + } + + @Override + public int filterAnd( + final Context context, + final Chunk[] inputChunks, + final int chunkSize, + final WritableBooleanChunk results) { + final Class[] paramTypes = ArgumentsSingular.buildParamTypes(inputChunks); + context.resultChunk.setSize(0); + // Count the values that changed from true to false + int count = 0; + for (int i = 0; i < chunkSize; ++i) { + boolean result = results.get(i); + // Save the cost of the call if the result is already false + if (!result) { + continue; + } + final Object[] params = ArgumentsSingular.buildArguments(inputChunks, i); + boolean newResult = function.call(boolean.class, CALL_METHOD, paramTypes, params); + results.set(i, newResult); + count += result == newResult ? 0 : 1; + } + return count; + } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/lang/impl/QueryLibraryImportsDefaults.java b/engine/table/src/main/java/io/deephaven/engine/table/lang/impl/QueryLibraryImportsDefaults.java index 5ef0ad1890e..d06ba6c2b8a 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/lang/impl/QueryLibraryImportsDefaults.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/lang/impl/QueryLibraryImportsDefaults.java @@ -79,6 +79,7 @@ public Set> classes() { LongSizedDataStructure.class, ConcurrentHashMap.class, Chunk.class, + BooleanChunk.class, ByteChunk.class, CharChunk.class, ShortChunk.class, @@ -88,6 +89,7 @@ public Set> classes() { DoubleChunk.class, ObjectChunk.class, WritableChunk.class, + WritableBooleanChunk.class, WritableByteChunk.class, WritableCharChunk.class, WritableShortChunk.class, diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java index 1dc18266e46..de502b44ae6 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/QueryTableWhereTest.java @@ -6,9 +6,7 @@ import io.deephaven.api.RawString; import io.deephaven.api.filter.Filter; import io.deephaven.base.verify.Assert; -import io.deephaven.chunk.Chunk; -import io.deephaven.chunk.LongChunk; -import io.deephaven.chunk.WritableLongChunk; +import io.deephaven.chunk.*; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.context.QueryScope; @@ -867,8 +865,10 @@ private TestChunkFilter(ChunkFilter actualFilter, long sleepDurationNanos) { } @Override - public void filter(Chunk values, LongChunk keys, - WritableLongChunk results) { + public void filter( + final Chunk values, + final LongChunk keys, + final WritableLongChunk results) { if (++invokes == 1) { latch.countDown(); } @@ -883,6 +883,42 @@ public void filter(Chunk values, LongChunk key actualFilter.filter(values, keys, results); } + @Override + public int filter( + final Chunk values, + final WritableBooleanChunk results) { + if (++invokes == 1) { + latch.countDown(); + } + invokedValues += values.size(); + if (sleepDurationNanos > 0) { + long nanos = sleepDurationNanos * values.size(); + final long timeStart = System.nanoTime(); + final long timeEnd = timeStart + nanos; + // noinspection StatementWithEmptyBody + while (System.nanoTime() < timeEnd); + } + return actualFilter.filter(values, results); + } + + @Override + public int filterAnd( + final Chunk values, + final WritableBooleanChunk results) { + if (++invokes == 1) { + latch.countDown(); + } + invokedValues += values.size(); + if (sleepDurationNanos > 0) { + long nanos = sleepDurationNanos * values.size(); + final long timeStart = System.nanoTime(); + final long timeEnd = timeStart + nanos; + // noinspection StatementWithEmptyBody + while (System.nanoTime() < timeEnd); + } + return actualFilter.filterAnd(values, results); + } + void reset() { invokes = invokedValues = 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 e917d44885f..9ecb8e14e42 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 @@ -9,6 +9,7 @@ import gnu.trove.set.hash.TIntHashSet; import io.deephaven.api.agg.Aggregation; import io.deephaven.api.agg.spec.AggSpec; +import io.deephaven.api.filter.Filter; import io.deephaven.api.object.UnionObject; import io.deephaven.base.verify.Assert; import io.deephaven.chunk.IntChunk; @@ -19,6 +20,8 @@ import io.deephaven.engine.table.ColumnSource; import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; +import io.deephaven.engine.table.impl.select.DynamicWhereFilter; +import io.deephaven.engine.table.impl.select.MatchPairFactory; import io.deephaven.engine.table.impl.util.ColumnHolder; import io.deephaven.engine.table.vectors.ColumnVectors; import io.deephaven.engine.testutil.*; @@ -119,7 +122,11 @@ public void testBy() { assertEquals(0, sums.get(0)); assertEquals(22 + 4, sums.get(1)); - Table doubleCounted = table.aggBy(List.of(AggCount("Count1"), AggCount("Count2")), "A"); + Table doubleCounted = table.aggBy( + List.of( + AggCount("Count1"), + AggCount("Count2")), + "A"); show(doubleCounted); assertEquals(2, doubleCounted.size()); @@ -178,6 +185,164 @@ public void testBy() { show(percentilesAll); } + @Test + public void testAggCountWhere() { + ColumnHolder aHolder = col("A", 0, 0, 1, 1, 0, 0, 1, 1, 0, 0); + ColumnHolder bHolder = col("B", 1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + ColumnHolder cHolder = col("C", 1, 1, 1, 1, 1, 1, 1, 1, 1, 1); + ColumnHolder dHolder = booleanCol("D", true, true, true, true, true, false, false, false, false, false); + final Instant startInstant = Instant.parse("1970-01-01T00:00:00.000Z"); + + final Instant[] instantData = new Instant[] { + startInstant, // 1970-01-01T00:00:00.000Z + startInstant.plusNanos(10 * 1_000_000), // 1970-01-01T00:00:00.010Z + startInstant.plusNanos(20 * 1_000_000), // 1970-01-01T00:00:00.020Z + startInstant.plusNanos(30 * 1_000_000), // 1970-01-01T00:00:00.030Z + startInstant.plusNanos(40 * 1_000_000), // 1970-01-01T00:00:00.040Z + startInstant.plusNanos(50 * 1_000_000), // 1970-01-01T00:00:00.050Z + startInstant.plusNanos(60 * 1_000_000), // 1970-01-01T00:00:00.060Z + startInstant.plusNanos(70 * 1_000_000), // 1970-01-01T00:00:00.070Z + startInstant.plusNanos(80 * 1_000_000), // 1970-01-01T00:00:00.080Z + startInstant.plusNanos(90 * 1_000_000) // 1970-01-01T00:00:00.090Z + }; + ColumnHolder eHolder = instantCol("E", instantData); + ColumnHolder fHolder = stringCol("F", "A", "B", "C", "D", "E", "A", "B", "C", "D", "E"); + + Table table = TableTools.newTable(aHolder, bHolder, cHolder, dHolder, eHolder, fHolder); + show(table); + assertEquals(10, table.size()); + assertEquals(2, table.groupBy("A").size()); + Table doubleCounted = table.aggBy( + List.of( + AggCountWhere("filter1", "B >= 5"), + AggCountWhere("filter2", "B >= 5", "B != 8"), + AggCountWhere("filter3", Filter.or(Filter.from("B >= 5", "B == 3"))), + AggCountWhere("filter4", "true"), + AggCountWhere("filter5", "false"), + AggCountWhere("filter6", "B % 2 == 0"), + AggCountWhere("filter7", Filter.and(Filter.or(Filter.from("false", "B % 3 == 0")), + Filter.or(Filter.from("false", "B % 2 == 0")))), + AggCountWhere("filter8", "B % 2 == 0", "B % 3 == 0"), + AggCountWhere("filter9", Filter.and(Filter.and(Filter.from("B > 0")), + Filter.and(Filter.from("B <= 10", "B >= 5")))), + // Multiple input columns + AggCountWhere("filter10", "B >= 5", "C == 1"), + AggCountWhere("filter11", "B >= 5 && C == 1 && A == 0"), + AggCountWhere("filter12", "B >= 5", "C >= 1"), + // Boolean column test + AggCountWhere("filter13", "D == true"), + // Instant column test + AggCountWhere("filter14", "E > '1970-01-01T00:00:00.030Z'"), + // String column test + AggCountWhere("filter15", "F != 'A'", "F != 'B'")), + "A"); + show(doubleCounted); + assertEquals(2, doubleCounted.size()); + + LongVector counts = ColumnVectors.ofLong(doubleCounted, "filter1"); + assertEquals(4L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter2"); + assertEquals(4L, counts.get(0)); + assertEquals(1L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter3"); + assertEquals(4L, counts.get(0)); + assertEquals(3L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter4"); + assertEquals(6L, counts.get(0)); + assertEquals(4L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter5"); + assertEquals(0L, counts.get(0)); + assertEquals(0L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter6"); + assertEquals(3L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter7"); + assertEquals(1L, counts.get(0)); + assertEquals(0L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter8"); + assertEquals(1L, counts.get(0)); + assertEquals(0L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter9"); + assertEquals(4L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter10"); + assertEquals(4L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter11"); + assertEquals(4L, counts.get(0)); + assertEquals(0L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter12"); + assertEquals(4L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter13"); + assertEquals(3L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter14"); + assertEquals(4L, counts.get(0)); + assertEquals(2L, counts.get(1)); + counts = ColumnVectors.ofLong(doubleCounted, "filter15"); + assertEquals(3L, counts.get(0)); + assertEquals(3L, counts.get(1)); + + doubleCounted = table.aggBy( + List.of( + AggCountWhere("filter1", "B >= 5"), + AggCountWhere("filter2", "B >= 5", "B != 8"), + AggCountWhere("filter3", Filter.or(Filter.from("B >= 5", "B == 3"))), + AggCountWhere("filter4", "true"), + AggCountWhere("filter5", "false"), + AggCountWhere("filter6", "B % 2 == 0"), + AggCountWhere("filter7", Filter.and(Filter.or(Filter.from("false", "B % 3 == 0")), + Filter.or(Filter.from("false", "B % 2 == 0")))), + AggCountWhere("filter8", "B % 2 == 0", "B % 3 == 0"), + AggCountWhere("filter9", Filter.and(Filter.and(Filter.from("B > 0")), + Filter.and(Filter.from("B <= 10", "B >= 5")))), + // Multiple input columns + AggCountWhere("filter10", "B >= 5", "C == 1"), + AggCountWhere("filter11", "B >= 5 && C == 1 && A == 0"), + AggCountWhere("filter12", "B >= 5", "C >= 1"), + // Boolean column test + AggCountWhere("filter13", "D == true"), + // Instant column test + AggCountWhere("filter14", "E > '1970-01-01T00:00:00.030Z'"), + // String column test + AggCountWhere("filter15", "F != 'A'", "F != 'B'"))); + show(doubleCounted); + assertEquals(1, doubleCounted.size()); + + counts = ColumnVectors.ofLong(doubleCounted, "filter1"); + assertEquals(6L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter2"); + assertEquals(5L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter3"); + assertEquals(7L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter4"); + assertEquals(10L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter5"); + assertEquals(0L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter6"); + assertEquals(5L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter7"); + assertEquals(1L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter8"); + assertEquals(1L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter9"); + assertEquals(6L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter10"); + assertEquals(6L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter11"); + assertEquals(4L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter12"); + assertEquals(6L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter13"); + assertEquals(5L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter14"); + assertEquals(6L, counts.get(0)); + counts = ColumnVectors.ofLong(doubleCounted, "filter15"); + assertEquals(6L, counts.get(0)); + } + @Test public void testComboByMinMaxTypes() { final Random random = new Random(0); @@ -192,7 +357,7 @@ public void testComboByMinMaxTypes() { new ShortGenerator(), new ByteGenerator(), new LongGenerator(), - new IntGenerator(10, 100), + new CharGenerator('A', 'z'), new SetGenerator<>(10.1, 20.1, 30.1), new FloatGenerator(0, 10.0f), new UnsortedInstantGenerator(DateTimeUtils.parseInstant("2020-03-17T12:00:00 NY"), @@ -240,6 +405,11 @@ private void testComboByIncremental(final String ctxt, final int size) { new SetGenerator<>(10.1, 20.1, 30.1), new SetGenerator<>(10.1, 20.1, 30.1, QueryConstants.NULL_DOUBLE))); + // Get a static set table for use in dynamic where filters + final QueryTable setTable = getTable(false, size / 10, random, + initColumnInfos(new String[] {"intCol"}, + new IntGenerator(10, 100))); + ExecutionContext.getContext().getQueryLibrary().importClass(TestAggBy.class); String[] groupByColumns = new String[0]; @@ -267,6 +437,61 @@ public Table e() { "Sym").sort("Sym"); } }, + // Bucketed AggCountWhere tests + new EvalNugget() { + public Table e() { + return queryTable.aggBy(List.of( + AggCountWhere("filter1", "intCol >= 50"), + AggCountWhere("filter2", "intCol >= 50", "intCol != 80"), + AggCountWhere("filter3", Filter.or(Filter.from("intCol >= 50", "intCol == 3"))), + AggCountWhere("filter4", "true"), + AggCountWhere("filter5", "false"), + AggCountWhere("filter6", "intCol % 2 == 0"), + AggCountWhere("filter7", + Filter.and(Filter.or(Filter.from("false", "intCol % 3 == 0")), + Filter.or(Filter.from("false", "intCol % 2 == 0")))), + AggCountWhere("filter8", "intCol % 2 == 0", "intCol % 3 == 0"), + AggCountWhere("filter9", + Filter.and(Filter.and(Filter.from("intCol > 0")), + Filter.and(Filter.from("intCol <= 10", "intCol >= 5")))), + // Multiple input columns + AggCountWhere("filter10", "intCol >= 5", "doubleCol <= 10.0"), + AggCountWhere("filter11", "intCol >= 5 && intColNulls != 3 && doubleCol <= 10.0"), + // DynamicWhereFilter + AggCountWhere("filter12", + new DynamicWhereFilter(setTable, true, + MatchPairFactory.getExpressions("intCol"))), + AggCountWhere("filter13", "doubleCol >= 5", "doubleColNulls >= 1")), + "Sym").sort("Sym"); + } + }, + // Zero-Key AggCountWhere tests + new EvalNugget() { + public Table e() { + return queryTable.aggBy(List.of( + AggCountWhere("filter1", "intCol >= 50"), + AggCountWhere("filter2", "intCol >= 50", "intCol != 80"), + AggCountWhere("filter3", Filter.or(Filter.from("intCol >= 50", "intCol == 3"))), + AggCountWhere("filter4", "true"), + AggCountWhere("filter5", "false"), + AggCountWhere("filter6", "intCol % 2 == 0"), + AggCountWhere("filter7", + Filter.and(Filter.or(Filter.from("false", "intCol % 3 == 0")), + Filter.or(Filter.from("false", "intCol % 2 == 0")))), + AggCountWhere("filter8", "intCol % 2 == 0", "intCol % 3 == 0"), + AggCountWhere("filter9", + Filter.and(Filter.and(Filter.from("intCol > 0")), + Filter.and(Filter.from("intCol <= 10", "intCol >= 5")))), + // Multiple input columns + AggCountWhere("filter10", "intCol >= 5", "doubleCol <= 10.0"), + AggCountWhere("filter11", "intCol >= 5 && intColNulls != 3 && doubleCol <= 10.0"), + // DynamicWhereFilter + AggCountWhere("filter12", + new DynamicWhereFilter(setTable, true, + MatchPairFactory.getExpressions("intCol"))), + AggCountWhere("filter13", "doubleCol >= 5", "doubleColNulls >= 1"))); + } + }, new QueryTableTest.TableComparator( queryTable.groupBy("Sym").view("Sym", "MinI=min(intCol)", "MinD=min(doubleCol)").sort("Sym"), "view", diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelArraySample.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelArraySample.java index 24432e01084..78ba79b13eb 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelArraySample.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelArraySample.java @@ -5,6 +5,7 @@ import java.lang.*; import java.util.*; import io.deephaven.base.string.cache.CompressedString; +import io.deephaven.chunk.BooleanChunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; @@ -14,6 +15,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.WritableBooleanChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableChunk; @@ -100,4 +102,36 @@ public LongChunk filter(Context __context, LongChunk __results) { + __results.setSize(__chunkSize); + int __count = 0; + for (int __my_i__ = 0; __my_i__ < __chunkSize; __my_i__++) { + final boolean __newResult = eq(v1_.size(), v2_.size()); + __results.set(__my_i__, __newResult); + // count every true value + __count += __newResult ? 1 : 0; + } + return __count; + } + + @Override + public int filterAnd(final Context __context, final Chunk[] __inputChunks, final int __chunkSize, final WritableBooleanChunk __results) { + __results.setSize(__chunkSize); + int __count = 0; + for (int __my_i__ = 0; __my_i__ < __chunkSize; __my_i__++) { + final boolean __result = __results.get(__my_i__); + if (!__result) { + // already false, no need to compute or increment the count + continue; + } + final boolean __newResult = eq(v1_.size(), v2_.size()); + __results.set(__my_i__, __newResult); + __results.set(__my_i__, __newResult); + // increment the count if the new result is TRUE + __count += __newResult ? 1 : 0; + } + return __count; + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelSample.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelSample.java index 1b57b3f40f2..9d13e977755 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelSample.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FilterKernelSample.java @@ -5,6 +5,7 @@ import java.lang.*; import java.util.*; import io.deephaven.base.string.cache.CompressedString; +import io.deephaven.chunk.BooleanChunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; @@ -14,6 +15,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.WritableBooleanChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableChunk; @@ -102,4 +104,44 @@ public LongChunk filter(Context __context, LongChunk __results) { + final ShortChunk __columnChunk0 = __inputChunks[0].asShortChunk(); + final DoubleChunk __columnChunk1 = __inputChunks[1].asDoubleChunk(); + __results.setSize(__chunkSize); + int __count = 0; + for (int __my_i__ = 0; __my_i__ < __chunkSize; __my_i__++) { + final short v1 = (short)__columnChunk0.get(__my_i__); + final double v2 = (double)__columnChunk1.get(__my_i__); + final boolean __newResult = "foo".equals((plus(plus(plus(p1, p2), v1), v2)) + p3); + __results.set(__my_i__, __newResult); + // count every true value + __count += __newResult ? 1 : 0; + } + return __count; + } + + @Override + public int filterAnd(final Context __context, final Chunk[] __inputChunks, final int __chunkSize, final WritableBooleanChunk __results) { + final ShortChunk __columnChunk0 = __inputChunks[0].asShortChunk(); + final DoubleChunk __columnChunk1 = __inputChunks[1].asDoubleChunk(); + __results.setSize(__chunkSize); + int __count = 0; + for (int __my_i__ = 0; __my_i__ < __chunkSize; __my_i__++) { + final boolean __result = __results.get(__my_i__); + if (!__result) { + // already false, no need to compute or increment the count + continue; + } + final short v1 = (short)__columnChunk0.get(__my_i__); + final double v2 = (double)__columnChunk1.get(__my_i__); + final boolean __newResult = "foo".equals((plus(plus(plus(p1, p2), v1), v2)) + p3); + __results.set(__my_i__, __newResult); + __results.set(__my_i__, __newResult); + // increment the count if the new result is TRUE + __count += __newResult ? 1 : 0; + } + return __count; + } +} diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaKernelSample.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaKernelSample.java index 28acdde3e30..733647f96d6 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaKernelSample.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaKernelSample.java @@ -5,6 +5,7 @@ import java.lang.*; import java.util.*; import io.deephaven.base.string.cache.CompressedString; +import io.deephaven.chunk.BooleanChunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; @@ -14,6 +15,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.WritableBooleanChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableChunk; diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaSample.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaSample.java index 7a027cde2ff..45035e32184 100644 --- a/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaSample.java +++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/select/FormulaSample.java @@ -5,6 +5,7 @@ import java.lang.*; import java.util.*; import io.deephaven.base.string.cache.CompressedString; +import io.deephaven.chunk.BooleanChunk; import io.deephaven.chunk.ByteChunk; import io.deephaven.chunk.CharChunk; import io.deephaven.chunk.Chunk; @@ -14,6 +15,7 @@ import io.deephaven.chunk.LongChunk; import io.deephaven.chunk.ObjectChunk; import io.deephaven.chunk.ShortChunk; +import io.deephaven.chunk.WritableBooleanChunk; import io.deephaven.chunk.WritableByteChunk; import io.deephaven.chunk.WritableCharChunk; import io.deephaven.chunk.WritableChunk; diff --git a/go/internal/proto/table/table.pb.go b/go/internal/proto/table/table.pb.go index 240ebd8af57..fe6f76ab686 100644 --- a/go/internal/proto/table/table.pb.go +++ b/go/internal/proto/table/table.pb.go @@ -3770,6 +3770,7 @@ type Aggregation struct { // *Aggregation_LastRowKey // *Aggregation_Partition // *Aggregation_Formula + // *Aggregation_CountWhere Type isAggregation_Type `protobuf_oneof:"type"` } @@ -3854,6 +3855,13 @@ func (x *Aggregation) GetFormula() *Aggregation_AggregationFormula { return nil } +func (x *Aggregation) GetCountWhere() *Aggregation_AggregationCountWhere { + if x, ok := x.GetType().(*Aggregation_CountWhere); ok { + return x.CountWhere + } + return nil +} + type isAggregation_Type interface { isAggregation_Type() } @@ -3882,6 +3890,10 @@ type Aggregation_Formula struct { Formula *Aggregation_AggregationFormula `protobuf:"bytes,6,opt,name=formula,proto3,oneof"` } +type Aggregation_CountWhere struct { + CountWhere *Aggregation_AggregationCountWhere `protobuf:"bytes,7,opt,name=count_where,json=countWhere,proto3,oneof"` +} + func (*Aggregation_Columns) isAggregation_Type() {} func (*Aggregation_Count) isAggregation_Type() {} @@ -3894,6 +3906,8 @@ func (*Aggregation_Partition) isAggregation_Type() {} func (*Aggregation_Formula) isAggregation_Type() {} +func (*Aggregation_CountWhere) isAggregation_Type() {} + type SortDescriptor struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -9026,6 +9040,62 @@ func (x *Aggregation_AggregationCount) GetColumnName() string { return "" } +type Aggregation_AggregationCountWhere struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The output column name + ColumnName string `protobuf:"bytes,1,opt,name=column_name,json=columnName,proto3" json:"column_name,omitempty"` + Filters []string `protobuf:"bytes,2,rep,name=filters,proto3" json:"filters,omitempty"` +} + +func (x *Aggregation_AggregationCountWhere) Reset() { + *x = Aggregation_AggregationCountWhere{} + if protoimpl.UnsafeEnabled { + mi := &file_deephaven_core_proto_table_proto_msgTypes[118] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Aggregation_AggregationCountWhere) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Aggregation_AggregationCountWhere) ProtoMessage() {} + +func (x *Aggregation_AggregationCountWhere) ProtoReflect() protoreflect.Message { + mi := &file_deephaven_core_proto_table_proto_msgTypes[118] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Aggregation_AggregationCountWhere.ProtoReflect.Descriptor instead. +func (*Aggregation_AggregationCountWhere) Descriptor() ([]byte, []int) { + return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 2} +} + +func (x *Aggregation_AggregationCountWhere) GetColumnName() string { + if x != nil { + return x.ColumnName + } + return "" +} + +func (x *Aggregation_AggregationCountWhere) GetFilters() []string { + if x != nil { + return x.Filters + } + return nil +} + type Aggregation_AggregationRowKey struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -9037,7 +9107,7 @@ type Aggregation_AggregationRowKey struct { func (x *Aggregation_AggregationRowKey) Reset() { *x = Aggregation_AggregationRowKey{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[118] + mi := &file_deephaven_core_proto_table_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9050,7 +9120,7 @@ func (x *Aggregation_AggregationRowKey) String() string { func (*Aggregation_AggregationRowKey) ProtoMessage() {} func (x *Aggregation_AggregationRowKey) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[118] + mi := &file_deephaven_core_proto_table_proto_msgTypes[119] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9063,7 +9133,7 @@ func (x *Aggregation_AggregationRowKey) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationRowKey.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationRowKey) Descriptor() ([]byte, []int) { - return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 2} + return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 3} } func (x *Aggregation_AggregationRowKey) GetColumnName() string { @@ -9085,7 +9155,7 @@ type Aggregation_AggregationPartition struct { func (x *Aggregation_AggregationPartition) Reset() { *x = Aggregation_AggregationPartition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[119] + mi := &file_deephaven_core_proto_table_proto_msgTypes[120] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9098,7 +9168,7 @@ func (x *Aggregation_AggregationPartition) String() string { func (*Aggregation_AggregationPartition) ProtoMessage() {} func (x *Aggregation_AggregationPartition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[119] + mi := &file_deephaven_core_proto_table_proto_msgTypes[120] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9111,7 +9181,7 @@ func (x *Aggregation_AggregationPartition) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationPartition.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationPartition) Descriptor() ([]byte, []int) { - return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 3} + return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 4} } func (x *Aggregation_AggregationPartition) GetColumnName() string { @@ -9139,7 +9209,7 @@ type Aggregation_AggregationFormula struct { func (x *Aggregation_AggregationFormula) Reset() { *x = Aggregation_AggregationFormula{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[120] + mi := &file_deephaven_core_proto_table_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9152,7 +9222,7 @@ func (x *Aggregation_AggregationFormula) String() string { func (*Aggregation_AggregationFormula) ProtoMessage() {} func (x *Aggregation_AggregationFormula) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[120] + mi := &file_deephaven_core_proto_table_proto_msgTypes[121] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9165,7 +9235,7 @@ func (x *Aggregation_AggregationFormula) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationFormula.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationFormula) Descriptor() ([]byte, []int) { - return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 4} + return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{37, 5} } func (x *Aggregation_AggregationFormula) GetSelectable() *Selectable { @@ -9187,7 +9257,7 @@ type RunChartDownsampleRequest_ZoomRange struct { func (x *RunChartDownsampleRequest_ZoomRange) Reset() { *x = RunChartDownsampleRequest_ZoomRange{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[121] + mi := &file_deephaven_core_proto_table_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9200,7 +9270,7 @@ func (x *RunChartDownsampleRequest_ZoomRange) String() string { func (*RunChartDownsampleRequest_ZoomRange) ProtoMessage() {} func (x *RunChartDownsampleRequest_ZoomRange) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[121] + mi := &file_deephaven_core_proto_table_proto_msgTypes[122] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9245,7 +9315,7 @@ type CreateInputTableRequest_InputTableKind struct { func (x *CreateInputTableRequest_InputTableKind) Reset() { *x = CreateInputTableRequest_InputTableKind{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[122] + mi := &file_deephaven_core_proto_table_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9258,7 +9328,7 @@ func (x *CreateInputTableRequest_InputTableKind) String() string { func (*CreateInputTableRequest_InputTableKind) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[122] + mi := &file_deephaven_core_proto_table_proto_msgTypes[123] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9337,7 +9407,7 @@ type CreateInputTableRequest_InputTableKind_InMemoryAppendOnly struct { func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) Reset() { *x = CreateInputTableRequest_InputTableKind_InMemoryAppendOnly{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[123] + mi := &file_deephaven_core_proto_table_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9350,7 +9420,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) String() str func (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[123] + mi := &file_deephaven_core_proto_table_proto_msgTypes[124] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9378,7 +9448,7 @@ type CreateInputTableRequest_InputTableKind_InMemoryKeyBacked struct { func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) Reset() { *x = CreateInputTableRequest_InputTableKind_InMemoryKeyBacked{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[124] + mi := &file_deephaven_core_proto_table_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9391,7 +9461,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) String() stri func (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[124] + mi := &file_deephaven_core_proto_table_proto_msgTypes[125] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9423,7 +9493,7 @@ type CreateInputTableRequest_InputTableKind_Blink struct { func (x *CreateInputTableRequest_InputTableKind_Blink) Reset() { *x = CreateInputTableRequest_InputTableKind_Blink{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[125] + mi := &file_deephaven_core_proto_table_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9436,7 +9506,7 @@ func (x *CreateInputTableRequest_InputTableKind_Blink) String() string { func (*CreateInputTableRequest_InputTableKind_Blink) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind_Blink) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[125] + mi := &file_deephaven_core_proto_table_proto_msgTypes[126] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9506,7 +9576,7 @@ type BatchTableRequest_Operation struct { func (x *BatchTableRequest_Operation) Reset() { *x = BatchTableRequest_Operation{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_core_proto_table_proto_msgTypes[126] + mi := &file_deephaven_core_proto_table_proto_msgTypes[127] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9519,7 +9589,7 @@ func (x *BatchTableRequest_Operation) String() string { func (*BatchTableRequest_Operation) ProtoMessage() {} func (x *BatchTableRequest_Operation) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_core_proto_table_proto_msgTypes[126] + mi := &file_deephaven_core_proto_table_proto_msgTypes[127] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -11473,7 +11543,7 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{ 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x22, 0x98, 0x08, 0x0a, 0x0b, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x6d, 0x6e, 0x73, 0x22, 0xd5, 0x09, 0x0a, 0x0b, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5d, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, @@ -11510,273 +11580,304 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{ 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x6f, 0x72, 0x6d, 0x75, - 0x6c, 0x61, 0x48, 0x00, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x1a, 0x75, 0x0a, - 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x12, 0x3e, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x2a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, - 0x70, 0x65, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x70, 0x61, 0x69, - 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x50, - 0x61, 0x69, 0x72, 0x73, 0x1a, 0x33, 0x0a, 0x10, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x34, 0x0a, 0x11, 0x41, 0x67, 0x67, - 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, 0x1f, - 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x1a, - 0x70, 0x0a, 0x14, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x6e, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x6e, 0x63, 0x6c, - 0x75, 0x64, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x1a, 0x63, 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x4d, 0x0a, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, - 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x0a, 0x73, 0x65, 0x6c, 0x65, - 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x84, - 0x02, 0x0a, 0x0e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, - 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x6c, 0x61, 0x48, 0x00, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x67, 0x0a, + 0x0b, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x77, 0x68, 0x65, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, + 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x1a, 0x75, 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x3e, 0x0a, 0x04, + 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x1f, 0x0a, 0x0b, + 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x70, 0x61, 0x69, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x0a, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x61, 0x69, 0x72, 0x73, 0x1a, 0x33, 0x0a, + 0x10, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, + 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x61, 0x62, 0x73, 0x6f, 0x6c, 0x75, 0x74, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x41, 0x62, 0x73, 0x6f, 0x6c, - 0x75, 0x74, 0x65, 0x12, 0x5d, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, - 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x0d, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, - 0x12, 0x17, 0x0a, 0x0a, 0x44, 0x45, 0x53, 0x43, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0xff, - 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x41, 0x53, 0x43, - 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x56, 0x45, - 0x52, 0x53, 0x45, 0x10, 0x02, 0x22, 0xf3, 0x01, 0x0a, 0x10, 0x53, 0x6f, 0x72, 0x74, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x49, 0x64, 0x12, 0x47, 0x0a, 0x05, 0x73, 0x6f, 0x72, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x6f, 0x72, 0x52, 0x05, 0x73, 0x6f, 0x72, 0x74, 0x73, 0x22, 0xf4, 0x01, 0x0a, 0x12, - 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, + 0x6d, 0x65, 0x1a, 0x52, 0x0a, 0x15, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, + 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x66, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x1a, 0x34, 0x0a, 0x11, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x70, 0x0a, 0x14, + 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x63, + 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x6f, 0x72, + 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x4d, 0x0a, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61, 0x62, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, + 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61, + 0x62, 0x6c, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x84, 0x02, 0x0a, 0x0e, + 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x1f, + 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x61, 0x62, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x41, 0x62, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x65, + 0x12, 0x5d, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x3f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, + 0x51, 0x0a, 0x0d, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x17, 0x0a, + 0x0a, 0x44, 0x45, 0x53, 0x43, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0xff, 0xff, 0xff, 0xff, + 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x41, 0x53, 0x43, 0x45, 0x4e, 0x44, + 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x56, 0x45, 0x52, 0x53, 0x45, + 0x10, 0x02, 0x22, 0xf3, 0x01, 0x0a, 0x10, 0x53, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, + 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, + 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, + 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, + 0x47, 0x0a, 0x05, 0x73, 0x6f, 0x72, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, + 0x72, 0x52, 0x05, 0x73, 0x6f, 0x72, 0x74, 0x73, 0x22, 0xf4, 0x01, 0x0a, 0x12, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, + 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, + 0xca, 0x02, 0x0a, 0x0e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, - 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0c, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x12, + 0x42, 0x02, 0x30, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x6f, + 0x77, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x49, 0x0a, 0x0a, 0x73, 0x65, 0x65, 0x6b, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, + 0x61, 0x6c, 0x52, 0x09, 0x73, 0x65, 0x65, 0x6b, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x20, 0x0a, + 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x65, 0x12, + 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x08, 0x52, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69, + 0x73, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x77, 0x61, 0x72, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x0a, 0x69, 0x73, 0x42, 0x61, 0x63, 0x6b, 0x77, 0x61, 0x72, 0x64, 0x22, 0x34, 0x0a, 0x0f, + 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, + 0x6f, 0x77, 0x22, 0x2c, 0x0a, 0x09, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, + 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, + 0x22, 0xd0, 0x01, 0x0a, 0x07, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x12, 0x23, 0x0a, 0x0c, + 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, 0x6c, + 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, + 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0a, 0x6c, 0x6f, 0x6e, 0x67, 0x5f, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x48, + 0x00, 0x52, 0x09, 0x6c, 0x6f, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2c, 0x0a, 0x0f, + 0x6e, 0x61, 0x6e, 0x6f, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x48, 0x00, 0x52, 0x0d, 0x6e, 0x61, 0x6e, + 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x22, 0xa5, 0x01, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x4c, 0x0a, + 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00, + 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x46, 0x0a, 0x07, 0x6c, + 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x07, 0x6c, 0x69, 0x74, 0x65, + 0x72, 0x61, 0x6c, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x82, 0x06, 0x0a, 0x09, + 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x03, 0x61, 0x6e, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6e, 0x64, 0x43, 0x6f, + 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6e, 0x64, 0x12, 0x40, + 0x0a, 0x02, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4f, + 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x02, 0x6f, 0x72, + 0x12, 0x43, 0x0a, 0x03, 0x6e, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, - 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, - 0x72, 0x73, 0x22, 0xca, 0x02, 0x0a, 0x0e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x63, 0x2e, 0x4e, 0x6f, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, + 0x52, 0x03, 0x6e, 0x6f, 0x74, 0x12, 0x4f, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x72, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x07, 0x63, + 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, 0x40, 0x0a, 0x02, 0x69, 0x6e, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x49, 0x6e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x02, 0x69, 0x6e, 0x12, 0x4c, 0x0a, 0x06, 0x69, 0x6e, 0x76, 0x6f, + 0x6b, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, - 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, - 0x6b, 0x65, 0x74, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, - 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x6f, 0x77, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x69, 0x6e, - 0x67, 0x52, 0x6f, 0x77, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x49, 0x0a, 0x0a, 0x73, 0x65, 0x65, 0x6b, 0x5f, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x69, - 0x74, 0x65, 0x72, 0x61, 0x6c, 0x52, 0x09, 0x73, 0x65, 0x65, 0x6b, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x12, 0x20, 0x0a, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x65, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, - 0x76, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x1f, - 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x77, 0x61, 0x72, 0x64, 0x18, 0x07, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x42, 0x61, 0x63, 0x6b, 0x77, 0x61, 0x72, 0x64, 0x22, - 0x34, 0x0a, 0x0f, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x72, 0x6f, 0x77, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x09, 0x72, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x52, 0x6f, 0x77, 0x22, 0x2c, 0x0a, 0x09, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, - 0x63, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, - 0x61, 0x6d, 0x65, 0x22, 0xd0, 0x01, 0x0a, 0x07, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x12, - 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, - 0x75, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, - 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, - 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0a, 0x6c, 0x6f, - 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, - 0x30, 0x01, 0x48, 0x00, 0x52, 0x09, 0x6c, 0x6f, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, - 0x2c, 0x0a, 0x0f, 0x6e, 0x61, 0x6e, 0x6f, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x48, 0x00, 0x52, 0x0d, - 0x6e, 0x61, 0x6e, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x07, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xa5, 0x01, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x12, 0x4c, 0x0a, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, - 0x65, 0x48, 0x00, 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x46, - 0x0a, 0x07, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x49, 0x6e, 0x76, + 0x6f, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x06, + 0x69, 0x6e, 0x76, 0x6f, 0x6b, 0x65, 0x12, 0x4d, 0x0a, 0x07, 0x69, 0x73, 0x5f, 0x6e, 0x75, 0x6c, + 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x49, 0x73, 0x4e, 0x75, + 0x6c, 0x6c, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x06, 0x69, + 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x12, 0x4f, 0x0a, 0x07, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, + 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x07, 0x6d, + 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x52, 0x0a, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, + 0x6e, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, + 0x74, 0x61, 0x69, 0x6e, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, + 0x52, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x06, 0x73, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, + 0x65, 0x61, 0x72, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, + 0x52, 0x06, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x22, 0x56, 0x0a, 0x0c, 0x41, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x55, 0x0a, 0x0b, 0x4f, 0x72, 0x43, 0x6f, + 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, + 0x54, 0x0a, 0x0c, 0x4e, 0x6f, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x44, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x07, 0x6c, - 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x82, - 0x06, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x03, - 0x61, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6e, - 0x64, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6e, - 0x64, 0x12, 0x40, 0x0a, 0x02, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, + 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x66, + 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, 0xd2, 0x03, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, + 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x62, 0x0a, 0x09, 0x6f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x44, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x4f, 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, - 0x02, 0x6f, 0x72, 0x12, 0x43, 0x0a, 0x03, 0x6e, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4e, 0x6f, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x48, 0x00, 0x52, 0x03, 0x6e, 0x6f, 0x74, 0x12, 0x4f, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x70, - 0x61, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x72, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, - 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x12, 0x40, 0x0a, 0x02, 0x69, 0x6e, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x49, 0x6e, 0x43, 0x6f, 0x6e, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x02, 0x69, 0x6e, 0x12, 0x4c, 0x0a, 0x06, 0x69, - 0x6e, 0x76, 0x6f, 0x6b, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x49, 0x6e, 0x76, 0x6f, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, - 0x00, 0x52, 0x06, 0x69, 0x6e, 0x76, 0x6f, 0x6b, 0x65, 0x12, 0x4d, 0x0a, 0x07, 0x69, 0x73, 0x5f, - 0x6e, 0x75, 0x6c, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, + 0x63, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5d, + 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x73, + 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52, 0x0f, 0x63, 0x61, + 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x3a, 0x0a, + 0x03, 0x6c, 0x68, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x49, - 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, - 0x52, 0x06, 0x69, 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x12, 0x4f, 0x0a, 0x07, 0x6d, 0x61, 0x74, 0x63, - 0x68, 0x65, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x61, - 0x74, 0x63, 0x68, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, - 0x52, 0x07, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x52, 0x0a, 0x08, 0x63, 0x6f, 0x6e, - 0x74, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x48, 0x00, 0x52, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x4c, 0x0a, - 0x06, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x48, 0x00, 0x52, 0x06, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x42, 0x06, 0x0a, 0x04, 0x64, - 0x61, 0x74, 0x61, 0x22, 0x56, 0x0a, 0x0c, 0x41, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0x55, 0x0a, 0x0b, 0x4f, - 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, - 0x72, 0x73, 0x22, 0x54, 0x0a, 0x0c, 0x4e, 0x6f, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x44, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, - 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x22, 0xd2, 0x03, 0x0a, 0x10, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x72, 0x65, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x62, 0x0a, - 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x44, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x03, 0x6c, 0x68, 0x73, 0x12, 0x3a, 0x0a, 0x03, 0x72, 0x68, 0x73, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x52, 0x03, 0x72, 0x68, 0x73, 0x22, 0x82, 0x01, 0x0a, 0x10, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, + 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x45, + 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x4c, 0x45, 0x53, + 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, + 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, + 0x4e, 0x10, 0x02, 0x12, 0x19, 0x0a, 0x15, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, + 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, 0x03, 0x12, 0x0a, + 0x0a, 0x06, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x53, 0x10, 0x04, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x4f, + 0x54, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x53, 0x10, 0x05, 0x22, 0xc5, 0x02, 0x0a, 0x0b, 0x49, + 0x6e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x06, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x48, 0x0a, 0x0a, + 0x63, 0x61, 0x6e, 0x64, 0x69, 0x64, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x43, 0x6f, 0x6e, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x72, 0x65, 0x4f, 0x70, 0x65, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x5d, 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, - 0x69, 0x76, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52, - 0x0f, 0x63, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, - 0x12, 0x3a, 0x0a, 0x03, 0x6c, 0x68, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x03, 0x6c, 0x68, 0x73, 0x12, 0x3a, 0x0a, 0x03, - 0x72, 0x68, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x63, 0x61, 0x6e, 0x64, + 0x69, 0x64, 0x61, 0x74, 0x65, 0x73, 0x12, 0x5d, 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, + 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, + 0x76, 0x69, 0x74, 0x79, 0x52, 0x0f, 0x63, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x4b, 0x0a, 0x0a, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x74, + 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, - 0x6c, 0x75, 0x65, 0x52, 0x03, 0x72, 0x68, 0x73, 0x22, 0x82, 0x01, 0x0a, 0x10, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x72, 0x65, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0d, 0x0a, - 0x09, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, - 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, - 0x41, 0x4c, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, - 0x54, 0x48, 0x41, 0x4e, 0x10, 0x02, 0x12, 0x19, 0x0a, 0x15, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, - 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, - 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x53, 0x10, 0x04, 0x12, 0x0e, 0x0a, - 0x0a, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x53, 0x10, 0x05, 0x22, 0xc5, 0x02, - 0x0a, 0x0b, 0x49, 0x6e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, - 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, - 0x48, 0x0a, 0x0a, 0x63, 0x61, 0x6e, 0x64, 0x69, 0x64, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, + 0x70, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x0f, 0x49, 0x6e, 0x76, 0x6f, 0x6b, 0x65, 0x43, 0x6f, 0x6e, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x40, + 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x12, 0x46, 0x0a, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x63, - 0x61, 0x6e, 0x64, 0x69, 0x64, 0x61, 0x74, 0x65, 0x73, 0x12, 0x5d, 0x0a, 0x10, 0x63, 0x61, 0x73, - 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, - 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52, 0x0f, 0x63, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, - 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x4b, 0x0a, 0x0a, 0x6d, 0x61, 0x74, 0x63, - 0x68, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x69, - 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x6d, 0x61, 0x74, 0x63, - 0x68, 0x54, 0x79, 0x70, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x0f, 0x49, 0x6e, 0x76, 0x6f, 0x6b, 0x65, - 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x6d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x6d, 0x65, 0x74, 0x68, 0x6f, - 0x64, 0x12, 0x40, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x74, 0x61, 0x72, - 0x67, 0x65, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x52, 0x09, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x5d, 0x0a, 0x0f, 0x49, - 0x73, 0x4e, 0x75, 0x6c, 0x6c, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4a, - 0x0a, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x09, 0x61, + 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x5d, 0x0a, 0x0f, 0x49, 0x73, 0x4e, 0x75, + 0x6c, 0x6c, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a, 0x09, 0x72, + 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x09, 0x72, 0x65, + 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x22, 0xa0, 0x02, 0x0a, 0x10, 0x4d, 0x61, 0x74, 0x63, + 0x68, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a, 0x09, + 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, + 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x09, 0x72, + 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x72, 0x65, 0x67, 0x65, + 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x65, 0x67, 0x65, 0x78, 0x12, 0x5d, + 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, + 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x61, 0x73, + 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x52, 0x0f, 0x63, 0x61, + 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x4b, 0x0a, + 0x0a, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, - 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x22, 0xa0, 0x02, 0x0a, 0x10, 0x4d, - 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x4a, 0x0a, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, - 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, - 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x72, - 0x65, 0x67, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x72, 0x65, 0x67, 0x65, - 0x78, 0x12, 0x5d, 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, + 0x09, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x22, 0xb0, 0x02, 0x0a, 0x11, 0x43, + 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x4a, 0x0a, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, + 0x65, 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x23, 0x0a, 0x0d, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x53, 0x74, 0x72, 0x69, 0x6e, + 0x67, 0x12, 0x5d, 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, @@ -11786,48 +11887,18 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{ 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, - 0x70, 0x65, 0x52, 0x09, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x22, 0xb0, 0x02, - 0x0a, 0x11, 0x43, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, - 0x65, 0x6e, 0x63, 0x65, 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, - 0x23, 0x0a, 0x0d, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x53, 0x74, - 0x72, 0x69, 0x6e, 0x67, 0x12, 0x5d, 0x0a, 0x10, 0x63, 0x61, 0x73, 0x65, 0x5f, 0x73, 0x65, 0x6e, - 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, - 0x74, 0x79, 0x52, 0x0f, 0x63, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, - 0x69, 0x74, 0x79, 0x12, 0x4b, 0x0a, 0x0a, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, - 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x61, 0x74, 0x63, - 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, - 0x22, 0x95, 0x01, 0x0a, 0x0f, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x64, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x73, - 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x65, 0x61, - 0x72, 0x63, 0x68, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x5d, 0x0a, 0x13, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, - 0x65, 0x6e, 0x63, 0x65, 0x52, 0x12, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x52, 0x65, - 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x0e, 0x46, 0x6c, 0x61, - 0x74, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x49, 0x64, 0x22, 0xaa, 0x01, 0x0a, 0x10, 0x4d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, + 0x70, 0x65, 0x52, 0x09, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x22, 0x95, 0x01, + 0x0a, 0x0f, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x73, 0x74, 0x72, 0x69, + 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, + 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x5d, 0x0a, 0x13, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, + 0x65, 0x52, 0x12, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x52, 0x65, 0x66, 0x65, 0x72, + 0x65, 0x6e, 0x63, 0x65, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x0e, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, @@ -11837,131 +11908,19 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{ 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, - 0x22, 0x97, 0x04, 0x0a, 0x19, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, - 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, - 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x69, 0x78, 0x65, 0x6c, 0x5f, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x70, 0x69, 0x78, - 0x65, 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x65, 0x0a, 0x0a, 0x7a, 0x6f, 0x6f, 0x6d, 0x5f, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, - 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x5a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, - 0x6e, 0x67, 0x65, 0x52, 0x09, 0x7a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x22, - 0x0a, 0x0d, 0x78, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x78, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x79, 0x43, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x8f, 0x01, 0x0a, 0x09, 0x5a, 0x6f, 0x6f, - 0x6d, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x2d, 0x0a, 0x0e, 0x6d, 0x69, 0x6e, 0x5f, 0x64, 0x61, - 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, - 0x30, 0x01, 0x48, 0x00, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6e, - 0x6f, 0x73, 0x88, 0x01, 0x01, 0x12, 0x2d, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x61, 0x74, - 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x30, - 0x01, 0x48, 0x01, 0x52, 0x0c, 0x6d, 0x61, 0x78, 0x44, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6e, 0x6f, - 0x73, 0x88, 0x01, 0x01, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74, - 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x61, 0x78, 0x5f, - 0x64, 0x61, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x22, 0xc3, 0x06, 0x0a, 0x17, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, - 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x5b, - 0x0a, 0x0f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, - 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x06, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x06, 0x73, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x5d, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x49, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, - 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, - 0x6b, 0x69, 0x6e, 0x64, 0x1a, 0xfb, 0x03, 0x0a, 0x0e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x91, 0x01, 0x0a, 0x15, 0x69, 0x6e, 0x5f, 0x6d, - 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, - 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, - 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, - 0x6e, 0x64, 0x2e, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70, 0x70, 0x65, 0x6e, - 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x48, 0x00, 0x52, 0x12, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, - 0x79, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x8e, 0x01, 0x0a, 0x14, - 0x69, 0x6e, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x62, 0x61, - 0x63, 0x6b, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5b, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x2e, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4b, 0x65, - 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x48, 0x00, 0x52, 0x11, 0x69, 0x6e, 0x4d, 0x65, 0x6d, - 0x6f, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x67, 0x0a, 0x05, - 0x62, 0x6c, 0x69, 0x6e, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4f, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x05, - 0x62, 0x6c, 0x69, 0x6e, 0x6b, 0x1a, 0x14, 0x0a, 0x12, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, - 0x79, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x1a, 0x34, 0x0a, 0x11, 0x49, - 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, - 0x12, 0x1f, 0x0a, 0x0b, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x6b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x1a, 0x07, 0x0a, 0x05, 0x42, 0x6c, 0x69, 0x6e, 0x6b, 0x42, 0x06, 0x0a, 0x04, 0x6b, 0x69, - 0x6e, 0x64, 0x42, 0x0c, 0x0a, 0x0a, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x22, 0xb8, 0x02, 0x0a, 0x0e, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, - 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4a, 0x0a, 0x07, 0x6c, - 0x65, 0x66, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, - 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, - 0x06, 0x6c, 0x65, 0x66, 0x74, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x69, 0x67, 0x68, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x72, 0x69, - 0x67, 0x68, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, - 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, - 0x64, 0x12, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, - 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x9c, 0x02, 0x0a, 0x17, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, - 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, - 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, - 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, - 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, - 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x31, 0x0a, 0x12, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x10, - 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, - 0x88, 0x01, 0x01, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, - 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0xa0, 0x02, 0x0a, 0x0c, 0x53, - 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, + 0x22, 0xaa, 0x01, 0x0a, 0x10, 0x4d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, + 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, + 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, + 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x22, 0x97, 0x04, + 0x0a, 0x19, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, + 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, @@ -11971,420 +11930,551 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{ 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x49, 0x64, 0x12, 0x3c, 0x0a, 0x18, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x73, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, - 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, - 0x65, 0x12, 0x3a, 0x0a, 0x17, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x65, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x15, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x6f, 0x73, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x22, 0xae, 0x1e, - 0x0a, 0x11, 0x42, 0x61, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x50, 0x0a, 0x03, 0x6f, 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x3e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x03, 0x6f, 0x70, 0x73, 0x1a, 0xc6, 0x1d, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x57, 0x0a, 0x0b, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x62, - 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x69, 0x78, 0x65, 0x6c, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x70, 0x69, 0x78, 0x65, 0x6c, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x65, 0x0a, 0x0a, 0x7a, 0x6f, 0x6f, 0x6d, 0x5f, 0x72, 0x61, 0x6e, + 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, - 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, - 0x52, 0x0a, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x54, 0x0a, 0x0a, - 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x75, 0x6e, + 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x5a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, 0x6e, 0x67, 0x65, + 0x52, 0x09, 0x7a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x22, 0x0a, 0x0d, 0x78, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x78, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x24, 0x0a, 0x0e, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x8f, 0x01, 0x0a, 0x09, 0x5a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, + 0x6e, 0x67, 0x65, 0x12, 0x2d, 0x0a, 0x0e, 0x6d, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, + 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x30, 0x01, 0x48, + 0x00, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x88, + 0x01, 0x01, 0x12, 0x2d, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x6e, + 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x30, 0x01, 0x48, 0x01, + 0x52, 0x0c, 0x6d, 0x61, 0x78, 0x44, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6e, 0x6f, 0x73, 0x88, 0x01, + 0x01, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x6e, + 0x61, 0x6e, 0x6f, 0x73, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x61, 0x74, + 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x22, 0xc3, 0x06, 0x0a, 0x17, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, + 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x5b, 0x0a, 0x0f, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, + 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x12, 0x5d, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x49, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0c, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, + 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, 0x6b, 0x69, 0x6e, + 0x64, 0x1a, 0xfb, 0x03, 0x0a, 0x0e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x91, 0x01, 0x0a, 0x15, 0x69, 0x6e, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, + 0x72, 0x79, 0x5f, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, + 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x2e, + 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, + 0x6c, 0x79, 0x48, 0x00, 0x52, 0x12, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70, + 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x8e, 0x01, 0x0a, 0x14, 0x69, 0x6e, 0x5f, + 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x65, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, + 0x6e, 0x64, 0x2e, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x42, 0x61, + 0x63, 0x6b, 0x65, 0x64, 0x48, 0x00, 0x52, 0x11, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x4b, 0x65, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x67, 0x0a, 0x05, 0x62, 0x6c, 0x69, + 0x6e, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, - 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x44, 0x72, 0x6f, - 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, - 0x00, 0x52, 0x0b, 0x64, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x52, - 0x0a, 0x06, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x75, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x12, 0x5b, 0x0a, 0x0b, 0x6c, 0x61, 0x7a, 0x79, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, + 0x69, 0x6e, 0x64, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x05, 0x62, 0x6c, 0x69, + 0x6e, 0x6b, 0x1a, 0x14, 0x0a, 0x12, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70, + 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x1a, 0x34, 0x0a, 0x11, 0x49, 0x6e, 0x4d, 0x65, + 0x6d, 0x6f, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x1f, 0x0a, + 0x0b, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0a, 0x6b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x07, + 0x0a, 0x05, 0x42, 0x6c, 0x69, 0x6e, 0x6b, 0x42, 0x06, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x42, + 0x0c, 0x0a, 0x0a, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xb8, 0x02, + 0x0a, 0x0e, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, + 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4a, 0x0a, 0x07, 0x6c, 0x65, 0x66, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x06, 0x6c, 0x65, + 0x66, 0x74, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x72, 0x69, 0x67, 0x68, 0x74, + 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64, 0x12, 0x28, + 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, + 0x63, 0x68, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x9c, 0x02, 0x0a, 0x17, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, - 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x48, 0x00, 0x52, 0x0a, 0x6c, 0x61, 0x7a, 0x79, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, - 0x4e, 0x0a, 0x04, 0x76, 0x69, 0x65, 0x77, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, + 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, + 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, + 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, + 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x31, 0x0a, + 0x12, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x6c, 0x69, + 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x10, 0x75, 0x6e, 0x69, + 0x71, 0x75, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x88, 0x01, 0x01, + 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0xa0, 0x02, 0x0a, 0x0c, 0x53, 0x6c, 0x69, 0x63, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, + 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, + 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, + 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, + 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, + 0x12, 0x3c, 0x0a, 0x18, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, 0x50, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x12, 0x3a, + 0x0a, 0x17, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x65, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x12, 0x42, + 0x02, 0x30, 0x01, 0x52, 0x15, 0x6c, 0x61, 0x73, 0x74, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x45, 0x78, 0x63, 0x6c, 0x75, 0x73, 0x69, 0x76, 0x65, 0x22, 0xae, 0x1e, 0x0a, 0x11, 0x42, + 0x61, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x50, 0x0a, 0x03, 0x6f, 0x70, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x76, 0x69, 0x65, 0x77, 0x12, - 0x5b, 0x0a, 0x0b, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x18, 0x07, + 0x63, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x6f, + 0x70, 0x73, 0x1a, 0xc6, 0x1d, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x57, 0x0a, 0x0b, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, + 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, + 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x65, + 0x6d, 0x70, 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x54, 0x0a, 0x0a, 0x74, 0x69, 0x6d, + 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, + 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, + 0x63, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, + 0x5a, 0x0a, 0x0c, 0x64, 0x72, 0x6f, 0x70, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, + 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, + 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, + 0x64, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x52, 0x0a, 0x06, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, + 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, + 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, + 0x5b, 0x0a, 0x0b, 0x6c, 0x61, 0x7a, 0x79, 0x5f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, - 0x52, 0x0a, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x69, 0x65, 0x77, 0x12, 0x52, 0x0a, 0x06, - 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, - 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x12, 0x63, 0x0a, 0x0f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, 0x64, 0x69, 0x73, 0x74, 0x69, - 0x6e, 0x63, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, - 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, - 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x4f, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, - 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, - 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x74, 0x0a, 0x13, 0x75, 0x6e, 0x73, 0x74, 0x72, 0x75, - 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x0b, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, - 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x12, 0x75, 0x6e, 0x73, 0x74, 0x72, 0x75, - 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x04, - 0x73, 0x6f, 0x72, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, + 0x52, 0x0a, 0x6c, 0x61, 0x7a, 0x79, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x4e, 0x0a, 0x04, + 0x76, 0x69, 0x65, 0x77, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, - 0x6f, 0x72, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, - 0x00, 0x52, 0x04, 0x73, 0x6f, 0x72, 0x74, 0x12, 0x4a, 0x0a, 0x04, 0x68, 0x65, 0x61, 0x64, 0x18, - 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, - 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x68, - 0x65, 0x61, 0x64, 0x12, 0x4a, 0x0a, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x76, 0x69, 0x65, 0x77, 0x12, 0x5b, 0x0a, 0x0b, + 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x18, 0x07, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x12, - 0x51, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x75, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x69, 0x65, 0x77, 0x12, 0x52, 0x0a, 0x06, 0x73, 0x65, 0x6c, + 0x65, 0x63, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x12, 0x63, 0x0a, + 0x0f, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x48, 0x00, 0x52, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, + 0x63, 0x74, 0x12, 0x4f, 0x0a, 0x06, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x66, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x12, 0x74, 0x0a, 0x13, 0x75, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, + 0x72, 0x65, 0x64, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, + 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x12, 0x75, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, + 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x04, 0x73, 0x6f, 0x72, + 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, + 0x73, 0x6f, 0x72, 0x74, 0x12, 0x4a, 0x0a, 0x04, 0x68, 0x65, 0x61, 0x64, 0x18, 0x0d, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, + 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x68, 0x65, 0x61, 0x64, + 0x12, 0x4a, 0x0a, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x12, 0x51, 0x0a, 0x07, + 0x68, 0x65, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, + 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, + 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, 0x42, 0x79, 0x12, + 0x51, 0x0a, 0x07, 0x74, 0x61, 0x69, 0x6c, 0x5f, 0x62, 0x79, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x42, - 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x68, 0x65, 0x61, 0x64, - 0x42, 0x79, 0x12, 0x51, 0x0a, 0x07, 0x74, 0x61, 0x69, 0x6c, 0x5f, 0x62, 0x79, 0x18, 0x10, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x74, 0x61, 0x69, 0x6c, + 0x42, 0x79, 0x12, 0x4d, 0x0a, 0x07, 0x75, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x11, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, - 0x69, 0x6c, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x74, - 0x61, 0x69, 0x6c, 0x42, 0x79, 0x12, 0x4d, 0x0a, 0x07, 0x75, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x75, 0x6e, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x12, 0x4d, 0x0a, 0x05, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x6d, 0x65, 0x72, 0x67, 0x65, + 0x12, 0x63, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x62, 0x6f, 0x5f, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, + 0x61, 0x74, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, + 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, + 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x4d, 0x0a, 0x07, 0x66, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, + 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x67, 0x72, 0x6f, - 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x75, 0x6e, 0x67, - 0x72, 0x6f, 0x75, 0x70, 0x12, 0x4d, 0x0a, 0x05, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x18, 0x12, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x6c, 0x61, 0x74, 0x74, + 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x66, 0x6c, 0x61, + 0x74, 0x74, 0x65, 0x6e, 0x12, 0x70, 0x0a, 0x14, 0x72, 0x75, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x72, + 0x74, 0x5f, 0x64, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x18, 0x16, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, + 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x48, 0x00, 0x52, 0x12, 0x72, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, + 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0a, 0x63, 0x72, 0x6f, 0x73, 0x73, 0x5f, + 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, + 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x63, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, + 0x69, 0x6e, 0x12, 0x60, 0x0a, 0x0c, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x5f, 0x6a, 0x6f, + 0x69, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4e, 0x61, 0x74, + 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, + 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5a, 0x0a, 0x0a, 0x65, 0x78, 0x61, 0x63, 0x74, 0x5f, 0x6a, 0x6f, + 0x69, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x61, + 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, + 0x12, 0x57, 0x0a, 0x09, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x1a, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x6d, 0x65, - 0x72, 0x67, 0x65, 0x12, 0x63, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x62, 0x6f, 0x5f, 0x61, 0x67, 0x67, - 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, - 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x6f, 0x41, - 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x4d, 0x0a, 0x07, 0x66, 0x6c, 0x61, 0x74, - 0x74, 0x65, 0x6e, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x6c, - 0x61, 0x74, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, - 0x66, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x12, 0x70, 0x0a, 0x14, 0x72, 0x75, 0x6e, 0x5f, 0x63, - 0x68, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x18, - 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, - 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x12, 0x72, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, - 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0a, 0x63, 0x72, 0x6f, - 0x73, 0x73, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, + 0x08, 0x6c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5c, 0x0a, 0x0a, 0x61, 0x73, 0x5f, + 0x6f, 0x66, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x63, 0x72, 0x6f, 0x73, - 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x60, 0x0a, 0x0c, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, - 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, + 0x63, 0x2e, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x08, 0x61, + 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x57, 0x0a, 0x0b, 0x66, 0x65, 0x74, 0x63, 0x68, + 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x66, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x12, 0x73, 0x0a, 0x15, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x5f, 0x70, 0x72, 0x65, 0x76, 0x69, 0x65, + 0x77, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, + 0x72, 0x70, 0x63, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, + 0x52, 0x13, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x6a, 0x0a, 0x12, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x5f, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, + 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, + 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x12, 0x51, 0x0a, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x62, 0x79, 0x18, 0x20, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x08, 0x75, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x12, 0x4e, 0x0a, 0x08, 0x77, 0x68, 0x65, 0x72, 0x65, 0x5f, 0x69, 0x6e, + 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x57, 0x68, 0x65, 0x72, 0x65, + 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x77, 0x68, 0x65, + 0x72, 0x65, 0x49, 0x6e, 0x12, 0x5d, 0x0a, 0x0d, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x65, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x6e, 0x61, 0x74, 0x75, - 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5a, 0x0a, 0x0a, 0x65, 0x78, 0x61, 0x63, 0x74, - 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, + 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, + 0x41, 0x6c, 0x6c, 0x12, 0x53, 0x0a, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, + 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x61, + 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x55, 0x0a, 0x08, 0x73, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, + 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, + 0x62, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x77, 0x68, 0x65, 0x6e, + 0x18, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, + 0x68, 0x65, 0x6e, 0x12, 0x54, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, 0x74, 0x61, 0x62, 0x6c, + 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65, 0x74, 0x61, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, + 0x6d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0a, 0x72, 0x61, 0x6e, + 0x67, 0x65, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, + 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, + 0x63, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x47, 0x0a, 0x02, 0x61, 0x6a, 0x18, 0x28, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x02, 0x61, 0x6a, 0x12, 0x49, + 0x0a, 0x03, 0x72, 0x61, 0x6a, 0x18, 0x29, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x61, 0x63, 0x74, 0x4a, - 0x6f, 0x69, 0x6e, 0x12, 0x57, 0x0a, 0x09, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, - 0x18, 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x65, 0x66, 0x74, 0x4a, - 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x48, 0x00, 0x52, 0x08, 0x6c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5c, 0x0a, 0x0a, - 0x61, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, - 0x52, 0x08, 0x61, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x57, 0x0a, 0x0b, 0x66, 0x65, - 0x74, 0x63, 0x68, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x6a, 0x12, 0x69, 0x0a, 0x11, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x18, 0x2a, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x48, 0x00, 0x52, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x12, 0x5a, 0x0a, 0x0a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x6a, 0x6f, + 0x69, 0x6e, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x75, 0x6c, + 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, + 0x12, 0x47, 0x0a, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x66, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x12, 0x73, 0x0a, 0x15, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x5f, 0x70, 0x72, 0x65, - 0x76, 0x69, 0x65, 0x77, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x1e, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x48, 0x00, 0x52, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x42, 0x04, 0x0a, 0x02, 0x6f, 0x70, 0x4a, + 0x04, 0x08, 0x14, 0x10, 0x15, 0x4a, 0x04, 0x08, 0x1d, 0x10, 0x1e, 0x2a, 0x62, 0x0a, 0x0f, 0x42, + 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x23, + 0x0a, 0x1f, 0x42, 0x41, 0x44, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56, + 0x49, 0x4f, 0x52, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x48, 0x52, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x09, + 0x0a, 0x05, 0x52, 0x45, 0x53, 0x45, 0x54, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x4b, 0x49, + 0x50, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x50, 0x4f, 0x49, 0x53, 0x4f, 0x4e, 0x10, 0x04, 0x2a, + 0x74, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4e, 0x75, 0x6c, 0x6c, 0x42, + 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x1b, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, + 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x55, 0x4c, 0x4c, + 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x53, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, + 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x53, 0x10, + 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x5a, 0x45, 0x52, 0x4f, 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, + 0x54, 0x45, 0x53, 0x10, 0x03, 0x2a, 0x1b, 0x0a, 0x09, 0x4e, 0x75, 0x6c, 0x6c, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, + 0x10, 0x00, 0x2a, 0x32, 0x0a, 0x0f, 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74, + 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x43, + 0x41, 0x53, 0x45, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x5f, + 0x43, 0x41, 0x53, 0x45, 0x10, 0x01, 0x2a, 0x26, 0x0a, 0x09, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x47, 0x55, 0x4c, 0x41, 0x52, 0x10, 0x00, + 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x32, 0xb9, + 0x32, 0x0a, 0x0c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, + 0x91, 0x01, 0x0a, 0x20, 0x47, 0x65, 0x74, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x1a, + 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, + 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x0a, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, 0x65, 0x76, - 0x69, 0x65, 0x77, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x48, 0x00, 0x52, 0x13, 0x61, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, 0x65, 0x76, 0x69, 0x65, - 0x77, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x6a, 0x0a, 0x12, 0x63, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x1f, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x98, 0x01, 0x0a, + 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, - 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x48, 0x00, 0x52, 0x10, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x12, 0x51, 0x0a, 0x09, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x62, - 0x79, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, + 0x65, 0x76, 0x69, 0x65, 0x77, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x0a, 0x45, 0x6d, 0x70, 0x74, + 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x84, 0x01, 0x0a, 0x09, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x33, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x44, 0x72, 0x6f, 0x70, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x08, 0x75, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x12, 0x4e, 0x0a, 0x08, 0x77, 0x68, 0x65, 0x72, 0x65, - 0x5f, 0x69, 0x6e, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x57, 0x68, - 0x65, 0x72, 0x65, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, - 0x77, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x12, 0x5d, 0x0a, 0x0d, 0x61, 0x67, 0x67, 0x72, 0x65, - 0x67, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x44, 0x72, 0x6f, 0x70, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x6c, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x41, 0x6c, 0x6c, 0x12, 0x53, 0x0a, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, - 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, - 0x52, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x55, 0x0a, 0x08, 0x73, - 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, + 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x06, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, - 0x6f, 0x74, 0x12, 0x62, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x77, - 0x68, 0x65, 0x6e, 0x18, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, - 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, - 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 0x12, 0x54, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, 0x74, - 0x61, 0x62, 0x6c, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, + 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8a, 0x01, 0x0a, 0x0a, + 0x4c, 0x61, 0x7a, 0x79, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, - 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, - 0x00, 0x52, 0x09, 0x6d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0a, - 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x72, - 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x47, 0x0a, 0x02, 0x61, 0x6a, 0x18, 0x28, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x02, 0x61, - 0x6a, 0x12, 0x49, 0x0a, 0x03, 0x72, 0x61, 0x6a, 0x18, 0x29, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x6a, 0x12, 0x69, 0x0a, 0x11, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, - 0x73, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, - 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, - 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x5a, 0x0a, 0x0a, 0x6d, 0x75, 0x6c, 0x74, 0x69, - 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4a, - 0x6f, 0x69, 0x6e, 0x12, 0x47, 0x0a, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x18, 0x2c, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, - 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x42, 0x04, 0x0a, 0x02, - 0x6f, 0x70, 0x4a, 0x04, 0x08, 0x14, 0x10, 0x15, 0x4a, 0x04, 0x08, 0x1d, 0x10, 0x1e, 0x2a, 0x62, - 0x0a, 0x0f, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, - 0x72, 0x12, 0x23, 0x0a, 0x1f, 0x42, 0x41, 0x44, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x42, 0x45, - 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x43, 0x49, - 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x48, 0x52, 0x4f, 0x57, 0x10, - 0x01, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x45, 0x53, 0x45, 0x54, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, - 0x53, 0x4b, 0x49, 0x50, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x50, 0x4f, 0x49, 0x53, 0x4f, 0x4e, - 0x10, 0x04, 0x2a, 0x74, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4e, 0x75, - 0x6c, 0x6c, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x1b, 0x4e, 0x55, - 0x4c, 0x4c, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, - 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, - 0x55, 0x4c, 0x4c, 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x53, 0x10, 0x01, 0x12, - 0x13, 0x0a, 0x0f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, 0x54, - 0x45, 0x53, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x5a, 0x45, 0x52, 0x4f, 0x5f, 0x44, 0x4f, 0x4d, - 0x49, 0x4e, 0x41, 0x54, 0x45, 0x53, 0x10, 0x03, 0x2a, 0x1b, 0x0a, 0x09, 0x4e, 0x75, 0x6c, 0x6c, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, 0x56, 0x41, - 0x4c, 0x55, 0x45, 0x10, 0x00, 0x2a, 0x32, 0x0a, 0x0f, 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, - 0x73, 0x69, 0x74, 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x41, 0x54, 0x43, - 0x48, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x49, 0x47, 0x4e, 0x4f, - 0x52, 0x45, 0x5f, 0x43, 0x41, 0x53, 0x45, 0x10, 0x01, 0x2a, 0x26, 0x0a, 0x09, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x47, 0x55, 0x4c, 0x41, - 0x52, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x45, 0x44, 0x10, - 0x01, 0x32, 0xb9, 0x32, 0x0a, 0x0c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x91, 0x01, 0x0a, 0x20, 0x47, 0x65, 0x74, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, - 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, - 0x65, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, - 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x0a, 0x46, 0x65, 0x74, 0x63, 0x68, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, + 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x04, 0x56, 0x69, 0x65, + 0x77, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x98, 0x01, 0x0a, 0x13, 0x41, 0x70, 0x70, 0x6c, 0x79, 0x50, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x8a, 0x01, 0x0a, 0x0a, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x69, 0x65, 0x77, 0x12, 0x38, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, + 0x06, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, - 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x70, 0x70, 0x6c, - 0x79, 0x50, 0x72, 0x65, 0x76, 0x69, 0x65, 0x77, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, - 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x0a, 0x45, - 0x6d, 0x70, 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x6d, - 0x70, 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x09, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, + 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x08, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x12, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x44, - 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x44, - 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x06, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, - 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8a, - 0x01, 0x0a, 0x0a, 0x4c, 0x61, 0x7a, 0x79, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x12, 0x38, 0x2e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x0e, 0x53, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x04, - 0x56, 0x69, 0x65, 0x77, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, - 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x8a, 0x01, 0x0a, 0x0a, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x69, 0x65, - 0x77, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x83, 0x01, 0x0a, 0x06, + 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, + 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, + 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x12, 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, + 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x73, + 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x86, 0x01, 0x0a, 0x06, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x4f, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, - 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x08, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x12, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, - 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8e, 0x01, - 0x0a, 0x0e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, - 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, - 0x6e, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, - 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x83, - 0x01, 0x0a, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x7f, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x80, 0x01, 0x0a, 0x04, 0x48, 0x65, 0x61, 0x64, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x12, 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, - 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x41, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, - 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, - 0x6f, 0x72, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, + 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x04, 0x48, 0x65, 0x61, 0x64, 0x12, 0x34, 0x2e, 0x69, + 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x04, 0x54, 0x61, 0x69, 0x6c, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, @@ -12392,266 +12482,258 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{ 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x04, 0x54, 0x61, 0x69, 0x6c, 0x12, - 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x06, 0x48, 0x65, - 0x61, 0x64, 0x42, 0x79, 0x12, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, - 0x61, 0x69, 0x6c, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, - 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x84, 0x01, 0x0a, 0x06, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x12, 0x36, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x06, 0x48, 0x65, 0x61, 0x64, 0x42, + 0x79, 0x12, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, + 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, + 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, + 0x0a, 0x06, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x12, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, + 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x07, 0x55, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, + 0x12, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x07, 0x55, 0x6e, 0x67, 0x72, - 0x6f, 0x75, 0x70, 0x12, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x4d, 0x65, 0x72, 0x67, + 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65, 0x72, 0x67, + 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, - 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x4d, - 0x65, 0x72, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, - 0x65, 0x72, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x6f, 0x73, 0x73, + 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, - 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, - 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, - 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x94, 0x01, 0x0a, 0x11, 0x4e, 0x61, 0x74, - 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x3b, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x94, 0x01, 0x0a, 0x11, 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, + 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x90, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, - 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x8e, 0x01, 0x0a, 0x0e, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, - 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x91, 0x01, 0x0a, 0x0e, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x73, 0x4f, 0x66, 0x4a, - 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, + 0x0f, 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, 0x12, 0x85, 0x01, 0x0a, 0x08, 0x41, 0x6a, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x86, 0x01, 0x0a, 0x09, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x35, 0x2e, + 0x8e, 0x01, 0x0a, 0x0e, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x73, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x91, 0x01, 0x0a, 0x0e, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x73, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x03, 0x88, 0x02, 0x01, 0x12, 0x85, 0x01, 0x0a, 0x08, 0x41, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, + 0x09, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, + 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, + 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x75, + 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x4d, 0x75, 0x6c, - 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x52, 0x61, 0x6e, + 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, + 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, - 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, - 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x91, 0x01, 0x0a, 0x0e, + 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x38, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, 0x12, + 0x8a, 0x01, 0x0a, 0x0c, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x6c, + 0x12, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x6c, + 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, + 0x09, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, - 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x91, - 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, - 0x65, 0x12, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, - 0x67, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, + 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, + 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, + 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x87, 0x01, 0x0a, 0x08, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x12, 0x37, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x61, 0x62, + 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, + 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8f, 0x01, + 0x0a, 0x0c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 0x12, 0x3b, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, 0x88, - 0x02, 0x01, 0x12, 0x8a, 0x01, 0x0a, 0x0c, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x41, 0x6c, 0x6c, 0x12, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, - 0x65, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x80, 0x01, 0x0a, 0x07, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x12, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x84, 0x01, 0x0a, 0x09, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x33, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, - 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, - 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, - 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x87, 0x01, 0x0a, 0x08, 0x53, 0x6e, 0x61, 0x70, 0x73, - 0x68, 0x6f, 0x74, 0x12, 0x37, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, + 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x12, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, + 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x3c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, + 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x75, + 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x92, 0x01, 0x0a, 0x10, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, + 0x12, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x8f, 0x01, 0x0a, 0x0c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, - 0x6e, 0x12, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, - 0x65, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x07, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x12, 0x31, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, - 0x70, 0x63, 0x2e, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, - 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x12, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, - 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x3c, 0x2e, 0x69, + 0x12, 0x80, 0x01, 0x0a, 0x07, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x12, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, - 0x2e, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, - 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, - 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x92, - 0x01, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, - 0x62, 0x6c, 0x65, 0x12, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2e, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x07, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x12, - 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x65, 0x22, 0x00, 0x12, 0x83, 0x01, 0x0a, 0x05, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x34, 0x2e, + 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, + 0x63, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x99, 0x01, 0x0a, 0x14, 0x45, 0x78, + 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x73, 0x12, 0x3e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, - 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x83, 0x01, 0x0a, 0x05, 0x42, 0x61, 0x74, 0x63, 0x68, - 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x61, 0x62, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, + 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, + 0x61, 0x62, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x72, 0x0a, 0x07, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, + 0x12, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, - 0x67, 0x72, 0x70, 0x63, 0x2e, 0x42, 0x61, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x09, 0x4d, 0x65, + 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, + 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65, 0x74, 0x61, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x99, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x3a, 0x2e, 0x69, + 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, + 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, + 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7c, 0x0a, 0x05, + 0x53, 0x6c, 0x69, 0x63, 0x65, 0x12, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, + 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, + 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x99, 0x01, 0x0a, - 0x14, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x73, 0x12, 0x3e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, - 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x72, 0x0a, 0x07, 0x53, 0x65, 0x65, 0x6b, - 0x52, 0x6f, 0x77, 0x12, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, - 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, - 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x65, 0x6b, 0x52, - 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, - 0x09, 0x4d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, - 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, - 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x99, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x43, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, - 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, - 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, - 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, - 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x7c, 0x0a, 0x05, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x12, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, - 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, - 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6c, 0x69, - 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, - 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, - 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x41, 0x48, - 0x01, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2f, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, - 0x76, 0x65, 0x6e, 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x67, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x41, 0x48, 0x01, 0x50, 0x01, + 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2f, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, + 0x2d, 0x63, 0x6f, 0x72, 0x65, 0x2f, 0x67, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -12667,7 +12749,7 @@ func file_deephaven_core_proto_table_proto_rawDescGZIP() []byte { } var file_deephaven_core_proto_table_proto_enumTypes = make([]protoimpl.EnumInfo, 12) -var file_deephaven_core_proto_table_proto_msgTypes = make([]protoimpl.MessageInfo, 127) +var file_deephaven_core_proto_table_proto_msgTypes = make([]protoimpl.MessageInfo, 128) var file_deephaven_core_proto_table_proto_goTypes = []interface{}{ (BadDataBehavior)(0), // 0: io.deephaven.proto.backplane.grpc.BadDataBehavior (UpdateByNullBehavior)(0), // 1: io.deephaven.proto.backplane.grpc.UpdateByNullBehavior @@ -12799,28 +12881,29 @@ var file_deephaven_core_proto_table_proto_goTypes = []interface{}{ (*AggSpec_AggSpecVar)(nil), // 127: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVar (*Aggregation_AggregationColumns)(nil), // 128: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns (*Aggregation_AggregationCount)(nil), // 129: io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount - (*Aggregation_AggregationRowKey)(nil), // 130: io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey - (*Aggregation_AggregationPartition)(nil), // 131: io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition - (*Aggregation_AggregationFormula)(nil), // 132: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula - (*RunChartDownsampleRequest_ZoomRange)(nil), // 133: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange - (*CreateInputTableRequest_InputTableKind)(nil), // 134: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind - (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly)(nil), // 135: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly - (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked)(nil), // 136: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked - (*CreateInputTableRequest_InputTableKind_Blink)(nil), // 137: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink - (*BatchTableRequest_Operation)(nil), // 138: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation - (*ticket.Ticket)(nil), // 139: io.deephaven.proto.backplane.grpc.Ticket + (*Aggregation_AggregationCountWhere)(nil), // 130: io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere + (*Aggregation_AggregationRowKey)(nil), // 131: io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey + (*Aggregation_AggregationPartition)(nil), // 132: io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition + (*Aggregation_AggregationFormula)(nil), // 133: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula + (*RunChartDownsampleRequest_ZoomRange)(nil), // 134: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange + (*CreateInputTableRequest_InputTableKind)(nil), // 135: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind + (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly)(nil), // 136: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly + (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked)(nil), // 137: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked + (*CreateInputTableRequest_InputTableKind_Blink)(nil), // 138: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink + (*BatchTableRequest_Operation)(nil), // 139: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation + (*ticket.Ticket)(nil), // 140: io.deephaven.proto.backplane.grpc.Ticket } var file_deephaven_core_proto_table_proto_depIdxs = []int32{ - 139, // 0: io.deephaven.proto.backplane.grpc.TableReference.ticket:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 0: io.deephaven.proto.backplane.grpc.TableReference.ticket:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 1: io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse.result_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 2: io.deephaven.proto.backplane.grpc.FetchTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 3: io.deephaven.proto.backplane.grpc.FetchTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 3: io.deephaven.proto.backplane.grpc.FetchTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 4: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 5: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 139, // 6: io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage.export_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 139, // 7: io.deephaven.proto.backplane.grpc.EmptyTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 139, // 8: io.deephaven.proto.backplane.grpc.TimeTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 139, // 9: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 5: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 6: io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage.export_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 7: io.deephaven.proto.backplane.grpc.EmptyTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 8: io.deephaven.proto.backplane.grpc.TimeTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 9: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 10: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 5, // 11: io.deephaven.proto.backplane.grpc.MathContext.rounding_mode:type_name -> io.deephaven.proto.backplane.grpc.MathContext.RoundingMode 77, // 12: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.ticks:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicks @@ -12832,61 +12915,61 @@ var file_deephaven_core_proto_table_proto_depIdxs = []int32{ 0, // 18: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_zero_delta_time:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior 22, // 19: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.big_value_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext 1, // 20: io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions.null_behavior:type_name -> io.deephaven.proto.backplane.grpc.UpdateByNullBehavior - 139, // 21: io.deephaven.proto.backplane.grpc.UpdateByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 21: io.deephaven.proto.backplane.grpc.UpdateByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 22: io.deephaven.proto.backplane.grpc.UpdateByRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 79, // 23: io.deephaven.proto.backplane.grpc.UpdateByRequest.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions 80, // 24: io.deephaven.proto.backplane.grpc.UpdateByRequest.operations:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation - 139, // 25: io.deephaven.proto.backplane.grpc.SelectDistinctRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 25: io.deephaven.proto.backplane.grpc.SelectDistinctRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 26: io.deephaven.proto.backplane.grpc.SelectDistinctRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 27: io.deephaven.proto.backplane.grpc.DropColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 27: io.deephaven.proto.backplane.grpc.DropColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 28: io.deephaven.proto.backplane.grpc.DropColumnsRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 29: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 29: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 30: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 31: io.deephaven.proto.backplane.grpc.HeadOrTailRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 31: io.deephaven.proto.backplane.grpc.HeadOrTailRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 32: io.deephaven.proto.backplane.grpc.HeadOrTailRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 33: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 33: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 34: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 35: io.deephaven.proto.backplane.grpc.UngroupRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 35: io.deephaven.proto.backplane.grpc.UngroupRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 36: io.deephaven.proto.backplane.grpc.UngroupRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 37: io.deephaven.proto.backplane.grpc.MergeTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 37: io.deephaven.proto.backplane.grpc.MergeTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 38: io.deephaven.proto.backplane.grpc.MergeTablesRequest.source_ids:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 39: io.deephaven.proto.backplane.grpc.SnapshotTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 39: io.deephaven.proto.backplane.grpc.SnapshotTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 40: io.deephaven.proto.backplane.grpc.SnapshotTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 41: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 41: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 42: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest.base_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 43: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest.trigger_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 44: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 44: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 45: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 46: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 47: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 47: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 48: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 49: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 50: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 50: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 51: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 52: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 53: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 53: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 54: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 55: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 56: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 56: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 57: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 58: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 6, // 59: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.as_of_match_rule:type_name -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.MatchRule - 139, // 60: io.deephaven.proto.backplane.grpc.AjRajTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 60: io.deephaven.proto.backplane.grpc.AjRajTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 61: io.deephaven.proto.backplane.grpc.AjRajTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 62: io.deephaven.proto.backplane.grpc.AjRajTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 63: io.deephaven.proto.backplane.grpc.MultiJoinInput.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 64: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 64: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 42, // 65: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest.multi_join_inputs:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinInput - 139, // 66: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 66: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 67: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 68: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 7, // 69: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.range_start_rule:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeStartRule 8, // 70: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.range_end_rule:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeEndRule 49, // 71: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.aggregations:type_name -> io.deephaven.proto.backplane.grpc.Aggregation - 139, // 72: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 72: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 73: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 104, // 74: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.aggregates:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate - 139, // 75: io.deephaven.proto.backplane.grpc.AggregateAllRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 75: io.deephaven.proto.backplane.grpc.AggregateAllRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 76: io.deephaven.proto.backplane.grpc.AggregateAllRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 47, // 77: io.deephaven.proto.backplane.grpc.AggregateAllRequest.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec 117, // 78: io.deephaven.proto.backplane.grpc.AggSpec.abs_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum @@ -12912,279 +12995,280 @@ var file_deephaven_core_proto_table_proto_depIdxs = []int32{ 116, // 98: io.deephaven.proto.backplane.grpc.AggSpec.weighted_avg:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted 116, // 99: io.deephaven.proto.backplane.grpc.AggSpec.weighted_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted 127, // 100: io.deephaven.proto.backplane.grpc.AggSpec.var:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVar - 139, // 101: io.deephaven.proto.backplane.grpc.AggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 140, // 101: io.deephaven.proto.backplane.grpc.AggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket 12, // 102: io.deephaven.proto.backplane.grpc.AggregateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 12, // 103: io.deephaven.proto.backplane.grpc.AggregateRequest.initial_groups_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference 49, // 104: io.deephaven.proto.backplane.grpc.AggregateRequest.aggregations:type_name -> io.deephaven.proto.backplane.grpc.Aggregation 128, // 105: io.deephaven.proto.backplane.grpc.Aggregation.columns:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns 129, // 106: io.deephaven.proto.backplane.grpc.Aggregation.count:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount - 130, // 107: io.deephaven.proto.backplane.grpc.Aggregation.first_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey - 130, // 108: io.deephaven.proto.backplane.grpc.Aggregation.last_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey - 131, // 109: io.deephaven.proto.backplane.grpc.Aggregation.partition:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition - 132, // 110: io.deephaven.proto.backplane.grpc.Aggregation.formula:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula - 10, // 111: io.deephaven.proto.backplane.grpc.SortDescriptor.direction:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection - 139, // 112: io.deephaven.proto.backplane.grpc.SortTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 113: io.deephaven.proto.backplane.grpc.SortTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 50, // 114: io.deephaven.proto.backplane.grpc.SortTableRequest.sorts:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor - 139, // 115: io.deephaven.proto.backplane.grpc.FilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 116: io.deephaven.proto.backplane.grpc.FilterTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 58, // 117: io.deephaven.proto.backplane.grpc.FilterTableRequest.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition - 139, // 118: io.deephaven.proto.backplane.grpc.SeekRowRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 56, // 119: io.deephaven.proto.backplane.grpc.SeekRowRequest.seek_value:type_name -> io.deephaven.proto.backplane.grpc.Literal - 55, // 120: io.deephaven.proto.backplane.grpc.Value.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 56, // 121: io.deephaven.proto.backplane.grpc.Value.literal:type_name -> io.deephaven.proto.backplane.grpc.Literal - 59, // 122: io.deephaven.proto.backplane.grpc.Condition.and:type_name -> io.deephaven.proto.backplane.grpc.AndCondition - 60, // 123: io.deephaven.proto.backplane.grpc.Condition.or:type_name -> io.deephaven.proto.backplane.grpc.OrCondition - 61, // 124: io.deephaven.proto.backplane.grpc.Condition.not:type_name -> io.deephaven.proto.backplane.grpc.NotCondition - 62, // 125: io.deephaven.proto.backplane.grpc.Condition.compare:type_name -> io.deephaven.proto.backplane.grpc.CompareCondition - 63, // 126: io.deephaven.proto.backplane.grpc.Condition.in:type_name -> io.deephaven.proto.backplane.grpc.InCondition - 64, // 127: io.deephaven.proto.backplane.grpc.Condition.invoke:type_name -> io.deephaven.proto.backplane.grpc.InvokeCondition - 65, // 128: io.deephaven.proto.backplane.grpc.Condition.is_null:type_name -> io.deephaven.proto.backplane.grpc.IsNullCondition - 66, // 129: io.deephaven.proto.backplane.grpc.Condition.matches:type_name -> io.deephaven.proto.backplane.grpc.MatchesCondition - 67, // 130: io.deephaven.proto.backplane.grpc.Condition.contains:type_name -> io.deephaven.proto.backplane.grpc.ContainsCondition - 68, // 131: io.deephaven.proto.backplane.grpc.Condition.search:type_name -> io.deephaven.proto.backplane.grpc.SearchCondition - 58, // 132: io.deephaven.proto.backplane.grpc.AndCondition.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition - 58, // 133: io.deephaven.proto.backplane.grpc.OrCondition.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition - 58, // 134: io.deephaven.proto.backplane.grpc.NotCondition.filter:type_name -> io.deephaven.proto.backplane.grpc.Condition - 11, // 135: io.deephaven.proto.backplane.grpc.CompareCondition.operation:type_name -> io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation - 3, // 136: io.deephaven.proto.backplane.grpc.CompareCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 57, // 137: io.deephaven.proto.backplane.grpc.CompareCondition.lhs:type_name -> io.deephaven.proto.backplane.grpc.Value - 57, // 138: io.deephaven.proto.backplane.grpc.CompareCondition.rhs:type_name -> io.deephaven.proto.backplane.grpc.Value - 57, // 139: io.deephaven.proto.backplane.grpc.InCondition.target:type_name -> io.deephaven.proto.backplane.grpc.Value - 57, // 140: io.deephaven.proto.backplane.grpc.InCondition.candidates:type_name -> io.deephaven.proto.backplane.grpc.Value - 3, // 141: io.deephaven.proto.backplane.grpc.InCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 4, // 142: io.deephaven.proto.backplane.grpc.InCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType - 57, // 143: io.deephaven.proto.backplane.grpc.InvokeCondition.target:type_name -> io.deephaven.proto.backplane.grpc.Value - 57, // 144: io.deephaven.proto.backplane.grpc.InvokeCondition.arguments:type_name -> io.deephaven.proto.backplane.grpc.Value - 55, // 145: io.deephaven.proto.backplane.grpc.IsNullCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 55, // 146: io.deephaven.proto.backplane.grpc.MatchesCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 3, // 147: io.deephaven.proto.backplane.grpc.MatchesCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 4, // 148: io.deephaven.proto.backplane.grpc.MatchesCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType - 55, // 149: io.deephaven.proto.backplane.grpc.ContainsCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 3, // 150: io.deephaven.proto.backplane.grpc.ContainsCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 4, // 151: io.deephaven.proto.backplane.grpc.ContainsCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType - 55, // 152: io.deephaven.proto.backplane.grpc.SearchCondition.optional_references:type_name -> io.deephaven.proto.backplane.grpc.Reference - 139, // 153: io.deephaven.proto.backplane.grpc.FlattenRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 154: io.deephaven.proto.backplane.grpc.FlattenRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 155: io.deephaven.proto.backplane.grpc.MetaTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 156: io.deephaven.proto.backplane.grpc.MetaTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 157: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 158: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 133, // 159: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.zoom_range:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange - 139, // 160: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 161: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.source_table_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 134, // 162: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.kind:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind - 139, // 163: io.deephaven.proto.backplane.grpc.WhereInRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 164: io.deephaven.proto.backplane.grpc.WhereInRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 12, // 165: io.deephaven.proto.backplane.grpc.WhereInRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 166: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 167: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 139, // 168: io.deephaven.proto.backplane.grpc.SliceRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 169: io.deephaven.proto.backplane.grpc.SliceRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 138, // 170: io.deephaven.proto.backplane.grpc.BatchTableRequest.ops:type_name -> io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation - 22, // 171: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions.math_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext - 81, // 172: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.column:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn - 82, // 173: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.spec:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec - 83, // 174: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.sum:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSum - 84, // 175: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMin - 85, // 176: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax - 86, // 177: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.product:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProduct - 87, // 178: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.fill:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFill - 88, // 179: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.ema:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma - 94, // 180: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_sum:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum - 95, // 181: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_group:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup - 96, // 182: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_avg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg - 97, // 183: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin - 98, // 184: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax - 99, // 185: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_product:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct - 93, // 186: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.delta:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta - 89, // 187: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.ems:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms - 90, // 188: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin - 91, // 189: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax - 92, // 190: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd - 100, // 191: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_count:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount - 101, // 192: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd - 102, // 193: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_wavg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg - 103, // 194: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_formula:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula - 24, // 195: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 23, // 196: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 24, // 197: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 23, // 198: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 24, // 199: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 23, // 200: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 24, // 201: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 23, // 202: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 24, // 203: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 23, // 204: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 25, // 205: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions - 23, // 206: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 207: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 208: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 209: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 210: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 211: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 212: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 213: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 214: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 215: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 216: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 217: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 218: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 219: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 220: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 221: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 222: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 223: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 224: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 225: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 9, // 226: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate.type:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType - 112, // 227: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted.columns:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn - 115, // 228: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique.non_unique_sentinel:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel - 2, // 229: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel.null_value:type_name -> io.deephaven.proto.backplane.grpc.NullValue - 47, // 230: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec - 21, // 231: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula.selectable:type_name -> io.deephaven.proto.backplane.grpc.Selectable - 135, // 232: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_append_only:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly - 136, // 233: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_key_backed:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked - 137, // 234: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.blink:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink - 18, // 235: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.empty_table:type_name -> io.deephaven.proto.backplane.grpc.EmptyTableRequest - 19, // 236: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.time_table:type_name -> io.deephaven.proto.backplane.grpc.TimeTableRequest - 28, // 237: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.drop_columns:type_name -> io.deephaven.proto.backplane.grpc.DropColumnsRequest - 20, // 238: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 239: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.lazy_update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 240: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 241: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 242: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 27, // 243: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select_distinct:type_name -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest - 52, // 244: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.filter:type_name -> io.deephaven.proto.backplane.grpc.FilterTableRequest - 29, // 245: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.unstructured_filter:type_name -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest - 51, // 246: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.sort:type_name -> io.deephaven.proto.backplane.grpc.SortTableRequest - 30, // 247: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 30, // 248: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 31, // 249: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 31, // 250: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 32, // 251: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.ungroup:type_name -> io.deephaven.proto.backplane.grpc.UngroupRequest - 33, // 252: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.merge:type_name -> io.deephaven.proto.backplane.grpc.MergeTablesRequest - 45, // 253: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.combo_aggregate:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest - 69, // 254: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.flatten:type_name -> io.deephaven.proto.backplane.grpc.FlattenRequest - 71, // 255: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.run_chart_downsample:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest - 36, // 256: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.cross_join:type_name -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest - 37, // 257: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.natural_join:type_name -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest - 38, // 258: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.exact_join:type_name -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest - 39, // 259: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.left_join:type_name -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest - 40, // 260: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.as_of_join:type_name -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest - 14, // 261: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.fetch_table:type_name -> io.deephaven.proto.backplane.grpc.FetchTableRequest - 15, // 262: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.apply_preview_columns:type_name -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest - 72, // 263: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.create_input_table:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest - 26, // 264: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_by:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest - 73, // 265: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.where_in:type_name -> io.deephaven.proto.backplane.grpc.WhereInRequest - 46, // 266: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate_all:type_name -> io.deephaven.proto.backplane.grpc.AggregateAllRequest - 48, // 267: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate:type_name -> io.deephaven.proto.backplane.grpc.AggregateRequest - 34, // 268: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot:type_name -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest - 35, // 269: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot_when:type_name -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest - 70, // 270: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.meta_table:type_name -> io.deephaven.proto.backplane.grpc.MetaTableRequest - 44, // 271: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.range_join:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest - 41, // 272: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 41, // 273: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.raj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 74, // 274: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.column_statistics:type_name -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest - 43, // 275: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.multi_join:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest - 75, // 276: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.slice:type_name -> io.deephaven.proto.backplane.grpc.SliceRequest - 139, // 277: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:input_type -> io.deephaven.proto.backplane.grpc.Ticket - 14, // 278: io.deephaven.proto.backplane.grpc.TableService.FetchTable:input_type -> io.deephaven.proto.backplane.grpc.FetchTableRequest - 15, // 279: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:input_type -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest - 18, // 280: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:input_type -> io.deephaven.proto.backplane.grpc.EmptyTableRequest - 19, // 281: io.deephaven.proto.backplane.grpc.TableService.TimeTable:input_type -> io.deephaven.proto.backplane.grpc.TimeTableRequest - 28, // 282: io.deephaven.proto.backplane.grpc.TableService.DropColumns:input_type -> io.deephaven.proto.backplane.grpc.DropColumnsRequest - 20, // 283: io.deephaven.proto.backplane.grpc.TableService.Update:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 284: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 285: io.deephaven.proto.backplane.grpc.TableService.View:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 286: io.deephaven.proto.backplane.grpc.TableService.UpdateView:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 287: io.deephaven.proto.backplane.grpc.TableService.Select:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 26, // 288: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:input_type -> io.deephaven.proto.backplane.grpc.UpdateByRequest - 27, // 289: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:input_type -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest - 52, // 290: io.deephaven.proto.backplane.grpc.TableService.Filter:input_type -> io.deephaven.proto.backplane.grpc.FilterTableRequest - 29, // 291: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:input_type -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest - 51, // 292: io.deephaven.proto.backplane.grpc.TableService.Sort:input_type -> io.deephaven.proto.backplane.grpc.SortTableRequest - 30, // 293: io.deephaven.proto.backplane.grpc.TableService.Head:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 30, // 294: io.deephaven.proto.backplane.grpc.TableService.Tail:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 31, // 295: io.deephaven.proto.backplane.grpc.TableService.HeadBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 31, // 296: io.deephaven.proto.backplane.grpc.TableService.TailBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 32, // 297: io.deephaven.proto.backplane.grpc.TableService.Ungroup:input_type -> io.deephaven.proto.backplane.grpc.UngroupRequest - 33, // 298: io.deephaven.proto.backplane.grpc.TableService.MergeTables:input_type -> io.deephaven.proto.backplane.grpc.MergeTablesRequest - 36, // 299: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:input_type -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest - 37, // 300: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:input_type -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest - 38, // 301: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:input_type -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest - 39, // 302: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:input_type -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest - 40, // 303: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:input_type -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest - 41, // 304: io.deephaven.proto.backplane.grpc.TableService.AjTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 41, // 305: io.deephaven.proto.backplane.grpc.TableService.RajTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 43, // 306: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:input_type -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest - 44, // 307: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:input_type -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest - 45, // 308: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:input_type -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest - 46, // 309: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:input_type -> io.deephaven.proto.backplane.grpc.AggregateAllRequest - 48, // 310: io.deephaven.proto.backplane.grpc.TableService.Aggregate:input_type -> io.deephaven.proto.backplane.grpc.AggregateRequest - 34, // 311: io.deephaven.proto.backplane.grpc.TableService.Snapshot:input_type -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest - 35, // 312: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:input_type -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest - 69, // 313: io.deephaven.proto.backplane.grpc.TableService.Flatten:input_type -> io.deephaven.proto.backplane.grpc.FlattenRequest - 71, // 314: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:input_type -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest - 72, // 315: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:input_type -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest - 73, // 316: io.deephaven.proto.backplane.grpc.TableService.WhereIn:input_type -> io.deephaven.proto.backplane.grpc.WhereInRequest - 76, // 317: io.deephaven.proto.backplane.grpc.TableService.Batch:input_type -> io.deephaven.proto.backplane.grpc.BatchTableRequest - 16, // 318: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:input_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest - 53, // 319: io.deephaven.proto.backplane.grpc.TableService.SeekRow:input_type -> io.deephaven.proto.backplane.grpc.SeekRowRequest - 70, // 320: io.deephaven.proto.backplane.grpc.TableService.MetaTable:input_type -> io.deephaven.proto.backplane.grpc.MetaTableRequest - 74, // 321: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:input_type -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest - 75, // 322: io.deephaven.proto.backplane.grpc.TableService.Slice:input_type -> io.deephaven.proto.backplane.grpc.SliceRequest - 13, // 323: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 324: io.deephaven.proto.backplane.grpc.TableService.FetchTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 325: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 326: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 327: io.deephaven.proto.backplane.grpc.TableService.TimeTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 328: io.deephaven.proto.backplane.grpc.TableService.DropColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 329: io.deephaven.proto.backplane.grpc.TableService.Update:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 330: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 331: io.deephaven.proto.backplane.grpc.TableService.View:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 332: io.deephaven.proto.backplane.grpc.TableService.UpdateView:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 333: io.deephaven.proto.backplane.grpc.TableService.Select:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 334: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 335: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 336: io.deephaven.proto.backplane.grpc.TableService.Filter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 337: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 338: io.deephaven.proto.backplane.grpc.TableService.Sort:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 339: io.deephaven.proto.backplane.grpc.TableService.Head:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 340: io.deephaven.proto.backplane.grpc.TableService.Tail:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 341: io.deephaven.proto.backplane.grpc.TableService.HeadBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 342: io.deephaven.proto.backplane.grpc.TableService.TailBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 343: io.deephaven.proto.backplane.grpc.TableService.Ungroup:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 344: io.deephaven.proto.backplane.grpc.TableService.MergeTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 345: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 346: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 347: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 348: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 349: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 350: io.deephaven.proto.backplane.grpc.TableService.AjTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 351: io.deephaven.proto.backplane.grpc.TableService.RajTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 352: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 353: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 354: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 355: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 356: io.deephaven.proto.backplane.grpc.TableService.Aggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 357: io.deephaven.proto.backplane.grpc.TableService.Snapshot:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 358: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 359: io.deephaven.proto.backplane.grpc.TableService.Flatten:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 360: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 361: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 362: io.deephaven.proto.backplane.grpc.TableService.WhereIn:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 363: io.deephaven.proto.backplane.grpc.TableService.Batch:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 17, // 364: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage - 54, // 365: io.deephaven.proto.backplane.grpc.TableService.SeekRow:output_type -> io.deephaven.proto.backplane.grpc.SeekRowResponse - 13, // 366: io.deephaven.proto.backplane.grpc.TableService.MetaTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 367: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 368: io.deephaven.proto.backplane.grpc.TableService.Slice:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 323, // [323:369] is the sub-list for method output_type - 277, // [277:323] is the sub-list for method input_type - 277, // [277:277] is the sub-list for extension type_name - 277, // [277:277] is the sub-list for extension extendee - 0, // [0:277] is the sub-list for field type_name + 131, // 107: io.deephaven.proto.backplane.grpc.Aggregation.first_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey + 131, // 108: io.deephaven.proto.backplane.grpc.Aggregation.last_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey + 132, // 109: io.deephaven.proto.backplane.grpc.Aggregation.partition:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition + 133, // 110: io.deephaven.proto.backplane.grpc.Aggregation.formula:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula + 130, // 111: io.deephaven.proto.backplane.grpc.Aggregation.count_where:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere + 10, // 112: io.deephaven.proto.backplane.grpc.SortDescriptor.direction:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection + 140, // 113: io.deephaven.proto.backplane.grpc.SortTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 114: io.deephaven.proto.backplane.grpc.SortTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 50, // 115: io.deephaven.proto.backplane.grpc.SortTableRequest.sorts:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor + 140, // 116: io.deephaven.proto.backplane.grpc.FilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 117: io.deephaven.proto.backplane.grpc.FilterTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 58, // 118: io.deephaven.proto.backplane.grpc.FilterTableRequest.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition + 140, // 119: io.deephaven.proto.backplane.grpc.SeekRowRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 56, // 120: io.deephaven.proto.backplane.grpc.SeekRowRequest.seek_value:type_name -> io.deephaven.proto.backplane.grpc.Literal + 55, // 121: io.deephaven.proto.backplane.grpc.Value.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference + 56, // 122: io.deephaven.proto.backplane.grpc.Value.literal:type_name -> io.deephaven.proto.backplane.grpc.Literal + 59, // 123: io.deephaven.proto.backplane.grpc.Condition.and:type_name -> io.deephaven.proto.backplane.grpc.AndCondition + 60, // 124: io.deephaven.proto.backplane.grpc.Condition.or:type_name -> io.deephaven.proto.backplane.grpc.OrCondition + 61, // 125: io.deephaven.proto.backplane.grpc.Condition.not:type_name -> io.deephaven.proto.backplane.grpc.NotCondition + 62, // 126: io.deephaven.proto.backplane.grpc.Condition.compare:type_name -> io.deephaven.proto.backplane.grpc.CompareCondition + 63, // 127: io.deephaven.proto.backplane.grpc.Condition.in:type_name -> io.deephaven.proto.backplane.grpc.InCondition + 64, // 128: io.deephaven.proto.backplane.grpc.Condition.invoke:type_name -> io.deephaven.proto.backplane.grpc.InvokeCondition + 65, // 129: io.deephaven.proto.backplane.grpc.Condition.is_null:type_name -> io.deephaven.proto.backplane.grpc.IsNullCondition + 66, // 130: io.deephaven.proto.backplane.grpc.Condition.matches:type_name -> io.deephaven.proto.backplane.grpc.MatchesCondition + 67, // 131: io.deephaven.proto.backplane.grpc.Condition.contains:type_name -> io.deephaven.proto.backplane.grpc.ContainsCondition + 68, // 132: io.deephaven.proto.backplane.grpc.Condition.search:type_name -> io.deephaven.proto.backplane.grpc.SearchCondition + 58, // 133: io.deephaven.proto.backplane.grpc.AndCondition.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition + 58, // 134: io.deephaven.proto.backplane.grpc.OrCondition.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition + 58, // 135: io.deephaven.proto.backplane.grpc.NotCondition.filter:type_name -> io.deephaven.proto.backplane.grpc.Condition + 11, // 136: io.deephaven.proto.backplane.grpc.CompareCondition.operation:type_name -> io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation + 3, // 137: io.deephaven.proto.backplane.grpc.CompareCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity + 57, // 138: io.deephaven.proto.backplane.grpc.CompareCondition.lhs:type_name -> io.deephaven.proto.backplane.grpc.Value + 57, // 139: io.deephaven.proto.backplane.grpc.CompareCondition.rhs:type_name -> io.deephaven.proto.backplane.grpc.Value + 57, // 140: io.deephaven.proto.backplane.grpc.InCondition.target:type_name -> io.deephaven.proto.backplane.grpc.Value + 57, // 141: io.deephaven.proto.backplane.grpc.InCondition.candidates:type_name -> io.deephaven.proto.backplane.grpc.Value + 3, // 142: io.deephaven.proto.backplane.grpc.InCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity + 4, // 143: io.deephaven.proto.backplane.grpc.InCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType + 57, // 144: io.deephaven.proto.backplane.grpc.InvokeCondition.target:type_name -> io.deephaven.proto.backplane.grpc.Value + 57, // 145: io.deephaven.proto.backplane.grpc.InvokeCondition.arguments:type_name -> io.deephaven.proto.backplane.grpc.Value + 55, // 146: io.deephaven.proto.backplane.grpc.IsNullCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference + 55, // 147: io.deephaven.proto.backplane.grpc.MatchesCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference + 3, // 148: io.deephaven.proto.backplane.grpc.MatchesCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity + 4, // 149: io.deephaven.proto.backplane.grpc.MatchesCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType + 55, // 150: io.deephaven.proto.backplane.grpc.ContainsCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference + 3, // 151: io.deephaven.proto.backplane.grpc.ContainsCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity + 4, // 152: io.deephaven.proto.backplane.grpc.ContainsCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType + 55, // 153: io.deephaven.proto.backplane.grpc.SearchCondition.optional_references:type_name -> io.deephaven.proto.backplane.grpc.Reference + 140, // 154: io.deephaven.proto.backplane.grpc.FlattenRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 155: io.deephaven.proto.backplane.grpc.FlattenRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 140, // 156: io.deephaven.proto.backplane.grpc.MetaTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 157: io.deephaven.proto.backplane.grpc.MetaTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 140, // 158: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 159: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 134, // 160: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.zoom_range:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange + 140, // 161: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 162: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.source_table_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 135, // 163: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.kind:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind + 140, // 164: io.deephaven.proto.backplane.grpc.WhereInRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 165: io.deephaven.proto.backplane.grpc.WhereInRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 12, // 166: io.deephaven.proto.backplane.grpc.WhereInRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 140, // 167: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 168: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 140, // 169: io.deephaven.proto.backplane.grpc.SliceRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 12, // 170: io.deephaven.proto.backplane.grpc.SliceRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference + 139, // 171: io.deephaven.proto.backplane.grpc.BatchTableRequest.ops:type_name -> io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation + 22, // 172: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions.math_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext + 81, // 173: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.column:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn + 82, // 174: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.spec:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec + 83, // 175: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.sum:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSum + 84, // 176: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMin + 85, // 177: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax + 86, // 178: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.product:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProduct + 87, // 179: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.fill:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFill + 88, // 180: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.ema:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma + 94, // 181: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_sum:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum + 95, // 182: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_group:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup + 96, // 183: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_avg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg + 97, // 184: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin + 98, // 185: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax + 99, // 186: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_product:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct + 93, // 187: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.delta:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta + 89, // 188: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.ems:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms + 90, // 189: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin + 91, // 190: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax + 92, // 191: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd + 100, // 192: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_count:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount + 101, // 193: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd + 102, // 194: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_wavg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg + 103, // 195: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_formula:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula + 24, // 196: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions + 23, // 197: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 24, // 198: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions + 23, // 199: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 24, // 200: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions + 23, // 201: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 24, // 202: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions + 23, // 203: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 24, // 204: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions + 23, // 205: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 25, // 206: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions + 23, // 207: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 208: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 209: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 210: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 211: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 212: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 213: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 214: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 215: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 216: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 217: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 218: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 219: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 220: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 221: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 222: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 223: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 224: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 225: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 23, // 226: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale + 9, // 227: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate.type:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType + 112, // 228: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted.columns:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn + 115, // 229: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique.non_unique_sentinel:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel + 2, // 230: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel.null_value:type_name -> io.deephaven.proto.backplane.grpc.NullValue + 47, // 231: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec + 21, // 232: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula.selectable:type_name -> io.deephaven.proto.backplane.grpc.Selectable + 136, // 233: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_append_only:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly + 137, // 234: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_key_backed:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked + 138, // 235: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.blink:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink + 18, // 236: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.empty_table:type_name -> io.deephaven.proto.backplane.grpc.EmptyTableRequest + 19, // 237: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.time_table:type_name -> io.deephaven.proto.backplane.grpc.TimeTableRequest + 28, // 238: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.drop_columns:type_name -> io.deephaven.proto.backplane.grpc.DropColumnsRequest + 20, // 239: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 240: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.lazy_update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 241: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 242: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 243: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 27, // 244: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select_distinct:type_name -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest + 52, // 245: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.filter:type_name -> io.deephaven.proto.backplane.grpc.FilterTableRequest + 29, // 246: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.unstructured_filter:type_name -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest + 51, // 247: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.sort:type_name -> io.deephaven.proto.backplane.grpc.SortTableRequest + 30, // 248: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest + 30, // 249: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest + 31, // 250: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest + 31, // 251: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest + 32, // 252: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.ungroup:type_name -> io.deephaven.proto.backplane.grpc.UngroupRequest + 33, // 253: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.merge:type_name -> io.deephaven.proto.backplane.grpc.MergeTablesRequest + 45, // 254: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.combo_aggregate:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest + 69, // 255: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.flatten:type_name -> io.deephaven.proto.backplane.grpc.FlattenRequest + 71, // 256: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.run_chart_downsample:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest + 36, // 257: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.cross_join:type_name -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest + 37, // 258: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.natural_join:type_name -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest + 38, // 259: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.exact_join:type_name -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest + 39, // 260: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.left_join:type_name -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest + 40, // 261: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.as_of_join:type_name -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest + 14, // 262: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.fetch_table:type_name -> io.deephaven.proto.backplane.grpc.FetchTableRequest + 15, // 263: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.apply_preview_columns:type_name -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest + 72, // 264: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.create_input_table:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest + 26, // 265: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_by:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest + 73, // 266: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.where_in:type_name -> io.deephaven.proto.backplane.grpc.WhereInRequest + 46, // 267: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate_all:type_name -> io.deephaven.proto.backplane.grpc.AggregateAllRequest + 48, // 268: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate:type_name -> io.deephaven.proto.backplane.grpc.AggregateRequest + 34, // 269: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot:type_name -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest + 35, // 270: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot_when:type_name -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest + 70, // 271: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.meta_table:type_name -> io.deephaven.proto.backplane.grpc.MetaTableRequest + 44, // 272: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.range_join:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest + 41, // 273: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest + 41, // 274: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.raj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest + 74, // 275: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.column_statistics:type_name -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest + 43, // 276: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.multi_join:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest + 75, // 277: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.slice:type_name -> io.deephaven.proto.backplane.grpc.SliceRequest + 140, // 278: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:input_type -> io.deephaven.proto.backplane.grpc.Ticket + 14, // 279: io.deephaven.proto.backplane.grpc.TableService.FetchTable:input_type -> io.deephaven.proto.backplane.grpc.FetchTableRequest + 15, // 280: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:input_type -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest + 18, // 281: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:input_type -> io.deephaven.proto.backplane.grpc.EmptyTableRequest + 19, // 282: io.deephaven.proto.backplane.grpc.TableService.TimeTable:input_type -> io.deephaven.proto.backplane.grpc.TimeTableRequest + 28, // 283: io.deephaven.proto.backplane.grpc.TableService.DropColumns:input_type -> io.deephaven.proto.backplane.grpc.DropColumnsRequest + 20, // 284: io.deephaven.proto.backplane.grpc.TableService.Update:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 285: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 286: io.deephaven.proto.backplane.grpc.TableService.View:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 287: io.deephaven.proto.backplane.grpc.TableService.UpdateView:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 20, // 288: io.deephaven.proto.backplane.grpc.TableService.Select:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest + 26, // 289: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:input_type -> io.deephaven.proto.backplane.grpc.UpdateByRequest + 27, // 290: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:input_type -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest + 52, // 291: io.deephaven.proto.backplane.grpc.TableService.Filter:input_type -> io.deephaven.proto.backplane.grpc.FilterTableRequest + 29, // 292: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:input_type -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest + 51, // 293: io.deephaven.proto.backplane.grpc.TableService.Sort:input_type -> io.deephaven.proto.backplane.grpc.SortTableRequest + 30, // 294: io.deephaven.proto.backplane.grpc.TableService.Head:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest + 30, // 295: io.deephaven.proto.backplane.grpc.TableService.Tail:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest + 31, // 296: io.deephaven.proto.backplane.grpc.TableService.HeadBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest + 31, // 297: io.deephaven.proto.backplane.grpc.TableService.TailBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest + 32, // 298: io.deephaven.proto.backplane.grpc.TableService.Ungroup:input_type -> io.deephaven.proto.backplane.grpc.UngroupRequest + 33, // 299: io.deephaven.proto.backplane.grpc.TableService.MergeTables:input_type -> io.deephaven.proto.backplane.grpc.MergeTablesRequest + 36, // 300: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:input_type -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest + 37, // 301: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:input_type -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest + 38, // 302: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:input_type -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest + 39, // 303: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:input_type -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest + 40, // 304: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:input_type -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest + 41, // 305: io.deephaven.proto.backplane.grpc.TableService.AjTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest + 41, // 306: io.deephaven.proto.backplane.grpc.TableService.RajTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest + 43, // 307: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:input_type -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest + 44, // 308: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:input_type -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest + 45, // 309: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:input_type -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest + 46, // 310: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:input_type -> io.deephaven.proto.backplane.grpc.AggregateAllRequest + 48, // 311: io.deephaven.proto.backplane.grpc.TableService.Aggregate:input_type -> io.deephaven.proto.backplane.grpc.AggregateRequest + 34, // 312: io.deephaven.proto.backplane.grpc.TableService.Snapshot:input_type -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest + 35, // 313: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:input_type -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest + 69, // 314: io.deephaven.proto.backplane.grpc.TableService.Flatten:input_type -> io.deephaven.proto.backplane.grpc.FlattenRequest + 71, // 315: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:input_type -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest + 72, // 316: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:input_type -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest + 73, // 317: io.deephaven.proto.backplane.grpc.TableService.WhereIn:input_type -> io.deephaven.proto.backplane.grpc.WhereInRequest + 76, // 318: io.deephaven.proto.backplane.grpc.TableService.Batch:input_type -> io.deephaven.proto.backplane.grpc.BatchTableRequest + 16, // 319: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:input_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest + 53, // 320: io.deephaven.proto.backplane.grpc.TableService.SeekRow:input_type -> io.deephaven.proto.backplane.grpc.SeekRowRequest + 70, // 321: io.deephaven.proto.backplane.grpc.TableService.MetaTable:input_type -> io.deephaven.proto.backplane.grpc.MetaTableRequest + 74, // 322: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:input_type -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest + 75, // 323: io.deephaven.proto.backplane.grpc.TableService.Slice:input_type -> io.deephaven.proto.backplane.grpc.SliceRequest + 13, // 324: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 325: io.deephaven.proto.backplane.grpc.TableService.FetchTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 326: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 327: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 328: io.deephaven.proto.backplane.grpc.TableService.TimeTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 329: io.deephaven.proto.backplane.grpc.TableService.DropColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 330: io.deephaven.proto.backplane.grpc.TableService.Update:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 331: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 332: io.deephaven.proto.backplane.grpc.TableService.View:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 333: io.deephaven.proto.backplane.grpc.TableService.UpdateView:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 334: io.deephaven.proto.backplane.grpc.TableService.Select:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 335: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 336: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 337: io.deephaven.proto.backplane.grpc.TableService.Filter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 338: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 339: io.deephaven.proto.backplane.grpc.TableService.Sort:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 340: io.deephaven.proto.backplane.grpc.TableService.Head:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 341: io.deephaven.proto.backplane.grpc.TableService.Tail:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 342: io.deephaven.proto.backplane.grpc.TableService.HeadBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 343: io.deephaven.proto.backplane.grpc.TableService.TailBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 344: io.deephaven.proto.backplane.grpc.TableService.Ungroup:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 345: io.deephaven.proto.backplane.grpc.TableService.MergeTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 346: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 347: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 348: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 349: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 350: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 351: io.deephaven.proto.backplane.grpc.TableService.AjTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 352: io.deephaven.proto.backplane.grpc.TableService.RajTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 353: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 354: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 355: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 356: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 357: io.deephaven.proto.backplane.grpc.TableService.Aggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 358: io.deephaven.proto.backplane.grpc.TableService.Snapshot:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 359: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 360: io.deephaven.proto.backplane.grpc.TableService.Flatten:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 361: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 362: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 363: io.deephaven.proto.backplane.grpc.TableService.WhereIn:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 364: io.deephaven.proto.backplane.grpc.TableService.Batch:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 17, // 365: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage + 54, // 366: io.deephaven.proto.backplane.grpc.TableService.SeekRow:output_type -> io.deephaven.proto.backplane.grpc.SeekRowResponse + 13, // 367: io.deephaven.proto.backplane.grpc.TableService.MetaTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 368: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 13, // 369: io.deephaven.proto.backplane.grpc.TableService.Slice:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse + 324, // [324:370] is the sub-list for method output_type + 278, // [278:324] is the sub-list for method input_type + 278, // [278:278] is the sub-list for extension type_name + 278, // [278:278] is the sub-list for extension extendee + 0, // [0:278] is the sub-list for field type_name } func init() { file_deephaven_core_proto_table_proto_init() } @@ -14610,7 +14694,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation_AggregationRowKey); i { + switch v := v.(*Aggregation_AggregationCountWhere); i { case 0: return &v.state case 1: @@ -14622,7 +14706,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation_AggregationPartition); i { + switch v := v.(*Aggregation_AggregationRowKey); i { case 0: return &v.state case 1: @@ -14634,7 +14718,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation_AggregationFormula); i { + switch v := v.(*Aggregation_AggregationPartition); i { case 0: return &v.state case 1: @@ -14646,7 +14730,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunChartDownsampleRequest_ZoomRange); i { + switch v := v.(*Aggregation_AggregationFormula); i { case 0: return &v.state case 1: @@ -14658,7 +14742,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind); i { + switch v := v.(*RunChartDownsampleRequest_ZoomRange); i { case 0: return &v.state case 1: @@ -14670,7 +14754,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly); i { + switch v := v.(*CreateInputTableRequest_InputTableKind); i { case 0: return &v.state case 1: @@ -14682,7 +14766,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked); i { + switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly); i { case 0: return &v.state case 1: @@ -14694,7 +14778,7 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[125].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind_Blink); i { + switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked); i { case 0: return &v.state case 1: @@ -14706,6 +14790,18 @@ func file_deephaven_core_proto_table_proto_init() { } } file_deephaven_core_proto_table_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateInputTableRequest_InputTableKind_Blink); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_deephaven_core_proto_table_proto_msgTypes[127].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BatchTableRequest_Operation); i { case 0: return &v.state @@ -14767,6 +14863,7 @@ func file_deephaven_core_proto_table_proto_init() { (*Aggregation_LastRowKey)(nil), (*Aggregation_Partition)(nil), (*Aggregation_Formula)(nil), + (*Aggregation_CountWhere)(nil), } file_deephaven_core_proto_table_proto_msgTypes[44].OneofWrappers = []interface{}{ (*Literal_StringValue)(nil), @@ -14841,13 +14938,13 @@ func file_deephaven_core_proto_table_proto_init() { (*AggSpec_AggSpecNonUniqueSentinel_ShortValue)(nil), (*AggSpec_AggSpecNonUniqueSentinel_CharValue)(nil), } - file_deephaven_core_proto_table_proto_msgTypes[121].OneofWrappers = []interface{}{} - file_deephaven_core_proto_table_proto_msgTypes[122].OneofWrappers = []interface{}{ + file_deephaven_core_proto_table_proto_msgTypes[122].OneofWrappers = []interface{}{} + file_deephaven_core_proto_table_proto_msgTypes[123].OneofWrappers = []interface{}{ (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly_)(nil), (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked_)(nil), (*CreateInputTableRequest_InputTableKind_Blink_)(nil), } - file_deephaven_core_proto_table_proto_msgTypes[126].OneofWrappers = []interface{}{ + file_deephaven_core_proto_table_proto_msgTypes[127].OneofWrappers = []interface{}{ (*BatchTableRequest_Operation_EmptyTable)(nil), (*BatchTableRequest_Operation_TimeTable)(nil), (*BatchTableRequest_Operation_DropColumns)(nil), @@ -14897,7 +14994,7 @@ func file_deephaven_core_proto_table_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_deephaven_core_proto_table_proto_rawDesc, NumEnums: 12, - NumMessages: 127, + NumMessages: 128, NumExtensions: 0, NumServices: 1, }, diff --git a/java-client/session-dagger/src/test/java/io/deephaven/client/AggBySessionTest.java b/java-client/session-dagger/src/test/java/io/deephaven/client/AggBySessionTest.java index 05238bf3bcd..9a38a262079 100644 --- a/java-client/session-dagger/src/test/java/io/deephaven/client/AggBySessionTest.java +++ b/java-client/session-dagger/src/test/java/io/deephaven/client/AggBySessionTest.java @@ -4,16 +4,8 @@ package io.deephaven.client; import io.deephaven.api.ColumnName; -import io.deephaven.api.agg.Aggregation; +import io.deephaven.api.agg.*; import io.deephaven.api.agg.Aggregation.Visitor; -import io.deephaven.api.agg.Aggregations; -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.Formula; -import io.deephaven.api.agg.LastRowKey; -import io.deephaven.api.agg.Partition; import io.deephaven.api.object.UnionObject; import io.deephaven.qst.table.TableSpec; import io.deephaven.qst.table.TimeTable; @@ -168,6 +160,11 @@ public void visit(Count count) { out.add(Count.of("MyCount")); } + @Override + public void visit(CountWhere countWhwere) { + out.add(CountWhere.of("CountWhere", "B > 0", "I <= 5")); + } + @Override public void visit(FirstRowKey firstRowKey) { out.add(FirstRowKey.of("First")); diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/AggregationBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/AggregationBuilder.java index 38e3679b561..97701859397 100644 --- a/java-client/session/src/main/java/io/deephaven/client/impl/AggregationBuilder.java +++ b/java-client/session/src/main/java/io/deephaven/client/impl/AggregationBuilder.java @@ -4,18 +4,13 @@ package io.deephaven.client.impl; import io.deephaven.api.Strings; -import io.deephaven.api.agg.Aggregations; -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.Formula; -import io.deephaven.api.agg.LastRowKey; +import io.deephaven.api.agg.*; import io.deephaven.api.Pair; -import io.deephaven.api.agg.Partition; +import io.deephaven.api.filter.Filter; import io.deephaven.proto.backplane.grpc.Aggregation; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount; +import io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey; @@ -79,6 +74,16 @@ public void visit(Count count) { .setColumnName(count.column().name()))); } + @Override + public void visit(CountWhere countWhere) { + final Collection filters = Filter.extractAnds(countWhere.filter()).stream() + .map(Strings::of) + .collect(Collectors.toList()); + out = singletonList(of(Builder::setCountWhere, AggregationCountWhere.newBuilder() + .setColumnName(countWhere.column().name()) + .addAllFilters(filters))); + } + @Override public void visit(FirstRowKey firstRowKey) { out = singletonList(of(Builder::setFirstRowKey, AggregationRowKey.newBuilder() diff --git a/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto b/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto index 5ca718773f0..7e89adf84bd 100644 --- a/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto +++ b/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto @@ -1055,6 +1055,13 @@ message Aggregation { string column_name = 1; } + message AggregationCountWhere { + // The output column name + string column_name = 1; + + repeated string filters = 2; + } + message AggregationRowKey { string column_name = 1; } @@ -1075,6 +1082,7 @@ message Aggregation { AggregationRowKey last_row_key = 4; AggregationPartition partition = 5; AggregationFormula formula = 6; + AggregationCountWhere count_where = 7; } } diff --git a/py/client/deephaven_core/proto/table_pb2.py b/py/client/deephaven_core/proto/table_pb2.py index 03c1ed76f9f..1442f9932a8 100644 --- a/py/client/deephaven_core/proto/table_pb2.py +++ b/py/client/deephaven_core/proto/table_pb2.py @@ -15,7 +15,7 @@ from deephaven_core.proto import ticket_pb2 as deephaven__core_dot_proto_dot_ticket__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n deephaven_core/proto/table.proto\x12!io.deephaven.proto.backplane.grpc\x1a!deephaven_core/proto/ticket.proto\"l\n\x0eTableReference\x12;\n\x06ticket\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.TicketH\x00\x12\x16\n\x0c\x62\x61tch_offset\x18\x02 \x01(\x11H\x00\x42\x05\n\x03ref\"\xc6\x01\n\x1d\x45xportedTableCreationResponse\x12\x44\n\tresult_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07success\x18\x02 \x01(\x08\x12\x12\n\nerror_info\x18\x03 \x01(\t\x12\x15\n\rschema_header\x18\x04 \x01(\x0c\x12\x11\n\tis_static\x18\x05 \x01(\x08\x12\x10\n\x04size\x18\x06 \x01(\x12\x42\x02\x30\x01\"\x97\x01\n\x11\x46\x65tchTableRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\xa0\x01\n\x1a\x41pplyPreviewColumnsRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\x1d\n\x1b\x45xportedTableUpdatesRequest\"\x8c\x01\n\x1a\x45xportedTableUpdateMessage\x12<\n\texport_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x1e\n\x16update_failure_message\x18\x03 \x01(\t\"c\n\x11\x45mptyTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\"\xef\x01\n\x10TimeTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x1e\n\x10start_time_nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1b\n\x11start_time_string\x18\x05 \x01(\tH\x00\x12\x1a\n\x0cperiod_nanos\x18\x03 \x01(\x12\x42\x02\x30\x01H\x01\x12\x17\n\rperiod_string\x18\x06 \x01(\tH\x01\x12\x13\n\x0b\x62link_table\x18\x04 \x01(\x08\x42\x0c\n\nstart_timeB\x08\n\x06period\"\xb1\x01\n\x15SelectOrUpdateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_specs\x18\x03 \x03(\t\"#\n\nSelectable\x12\r\n\x03raw\x18\x01 \x01(\tH\x00\x42\x06\n\x04type\"\x8c\x02\n\x0bMathContext\x12\x11\n\tprecision\x18\x01 \x01(\x11\x12R\n\rrounding_mode\x18\x02 \x01(\x0e\x32;.io.deephaven.proto.backplane.grpc.MathContext.RoundingMode\"\x95\x01\n\x0cRoundingMode\x12\x1f\n\x1bROUNDING_MODE_NOT_SPECIFIED\x10\x00\x12\x06\n\x02UP\x10\x01\x12\x08\n\x04\x44OWN\x10\x02\x12\x0b\n\x07\x43\x45ILING\x10\x03\x12\t\n\x05\x46LOOR\x10\x04\x12\x0b\n\x07HALF_UP\x10\x05\x12\r\n\tHALF_DOWN\x10\x06\x12\r\n\tHALF_EVEN\x10\x07\x12\x0f\n\x0bUNNECESSARY\x10\x08\"\xdb\x02\n\x13UpdateByWindowScale\x12[\n\x05ticks\x18\x01 \x01(\x0b\x32J.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicksH\x00\x12Y\n\x04time\x18\x02 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTimeH\x00\x1a$\n\x13UpdateByWindowTicks\x12\r\n\x05ticks\x18\x01 \x01(\x01\x1a^\n\x12UpdateByWindowTime\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x13\n\x05nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x19\n\x0f\x64uration_string\x18\x03 \x01(\tH\x00\x42\x08\n\x06windowB\x06\n\x04type\"\xe1\x03\n\x11UpdateByEmOptions\x12I\n\ron_null_value\x18\x01 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_nan_value\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_null_time\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12R\n\x16on_negative_delta_time\x18\x04 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12N\n\x12on_zero_delta_time\x18\x05 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12I\n\x11\x62ig_value_context\x18\x06 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContext\"f\n\x14UpdateByDeltaOptions\x12N\n\rnull_behavior\x18\x01 \x01(\x0e\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByNullBehavior\"\x9b\x37\n\x0fUpdateByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12S\n\x07options\x18\x03 \x01(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions\x12X\n\noperations\x18\x04 \x03(\x0b\x32\x44.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation\x12\x18\n\x10group_by_columns\x18\x05 \x03(\t\x1a\xc3\x03\n\x0fUpdateByOptions\x12\x1c\n\x0fuse_redirection\x18\x01 \x01(\x08H\x00\x88\x01\x01\x12\x1b\n\x0e\x63hunk_capacity\x18\x02 \x01(\x05H\x01\x88\x01\x01\x12.\n!max_static_sparse_memory_overhead\x18\x03 \x01(\x01H\x02\x88\x01\x01\x12$\n\x17initial_hash_table_size\x18\x04 \x01(\x05H\x03\x88\x01\x01\x12 \n\x13maximum_load_factor\x18\x05 \x01(\x01H\x04\x88\x01\x01\x12\x1f\n\x12target_load_factor\x18\x06 \x01(\x01H\x05\x88\x01\x01\x12\x44\n\x0cmath_context\x18\x07 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContextB\x12\n\x10_use_redirectionB\x11\n\x0f_chunk_capacityB$\n\"_max_static_sparse_memory_overheadB\x1a\n\x18_initial_hash_table_sizeB\x16\n\x14_maximum_load_factorB\x15\n\x13_target_load_factor\x1a\xf4\x30\n\x11UpdateByOperation\x12\x65\n\x06\x63olumn\x18\x01 \x01(\x0b\x32S.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumnH\x00\x1a\xef/\n\x0eUpdateByColumn\x12n\n\x04spec\x18\x01 \x01(\x0b\x32`.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\xd7.\n\x0cUpdateBySpec\x12\x85\x01\n\x03sum\x18\x01 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSumH\x00\x12\x85\x01\n\x03min\x18\x02 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMinH\x00\x12\x85\x01\n\x03max\x18\x03 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMaxH\x00\x12\x8d\x01\n\x07product\x18\x04 \x01(\x0b\x32z.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProductH\x00\x12}\n\x04\x66ill\x18\x05 \x01(\x0b\x32m.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFillH\x00\x12{\n\x03\x65ma\x18\x06 \x01(\x0b\x32l.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmaH\x00\x12\x8a\x01\n\x0brolling_sum\x18\x07 \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSumH\x00\x12\x8e\x01\n\rrolling_group\x18\x08 \x01(\x0b\x32u.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroupH\x00\x12\x8a\x01\n\x0brolling_avg\x18\t \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvgH\x00\x12\x8a\x01\n\x0brolling_min\x18\n \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMinH\x00\x12\x8a\x01\n\x0brolling_max\x18\x0b \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMaxH\x00\x12\x92\x01\n\x0frolling_product\x18\x0c \x01(\x0b\x32w.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProductH\x00\x12\x7f\n\x05\x64\x65lta\x18\r \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDeltaH\x00\x12{\n\x03\x65ms\x18\x0e \x01(\x0b\x32l.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmsH\x00\x12\x80\x01\n\x06\x65m_min\x18\x0f \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMinH\x00\x12\x80\x01\n\x06\x65m_max\x18\x10 \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMaxH\x00\x12\x80\x01\n\x06\x65m_std\x18\x11 \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStdH\x00\x12\x8e\x01\n\rrolling_count\x18\x12 \x01(\x0b\x32u.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountH\x00\x12\x8a\x01\n\x0brolling_std\x18\x13 \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStdH\x00\x12\x8c\x01\n\x0crolling_wavg\x18\x14 \x01(\x0b\x32t.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvgH\x00\x12\x92\x01\n\x0frolling_formula\x18\x15 \x01(\x0b\x32w.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormulaH\x00\x1a\x17\n\x15UpdateByCumulativeSum\x1a\x17\n\x15UpdateByCumulativeMin\x1a\x17\n\x15UpdateByCumulativeMax\x1a\x1b\n\x19UpdateByCumulativeProduct\x1a\x0e\n\x0cUpdateByFill\x1a\xa2\x01\n\x0bUpdateByEma\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa2\x01\n\x0bUpdateByEms\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmMin\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmMax\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmStd\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1aY\n\rUpdateByDelta\x12H\n\x07options\x18\x01 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions\x1a\xc0\x01\n\x12UpdateByRollingSum\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc2\x01\n\x14UpdateByRollingGroup\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingAvg\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingMin\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingMax\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc4\x01\n\x16UpdateByRollingProduct\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc2\x01\n\x14UpdateByRollingCount\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingStd\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xd8\x01\n\x13UpdateByRollingWAvg\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12\x15\n\rweight_column\x18\x03 \x01(\t\x1a\xea\x01\n\x16UpdateByRollingFormula\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12\x0f\n\x07\x66ormula\x18\x03 \x01(\t\x12\x13\n\x0bparam_token\x18\x04 \x01(\tB\x06\n\x04typeB\x06\n\x04type\"\xb1\x01\n\x15SelectDistinctRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xae\x01\n\x12\x44ropColumnsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xb5\x01\n\x1eUnstructuredFilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07\x66ilters\x18\x03 \x03(\t\"\xad\x01\n\x11HeadOrTailRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\"\xce\x01\n\x13HeadOrTailByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\x12\x1d\n\x15group_by_column_specs\x18\x04 \x03(\t\"\xc3\x01\n\x0eUngroupRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x11\n\tnull_fill\x18\x03 \x01(\x08\x12\x1a\n\x12\x63olumns_to_ungroup\x18\x04 \x03(\t\"\xad\x01\n\x12MergeTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x45\n\nsource_ids\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x12\n\nkey_column\x18\x03 \x01(\t\"\x9a\x01\n\x14SnapshotTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb1\x02\n\x18SnapshotWhenTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07\x62\x61se_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x45\n\ntrigger_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07initial\x18\x04 \x01(\x08\x12\x13\n\x0bincremental\x18\x05 \x01(\x08\x12\x0f\n\x07history\x18\x06 \x01(\x08\x12\x15\n\rstamp_columns\x18\x07 \x03(\t\"\xa7\x02\n\x16\x43rossJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\x14\n\x0creserve_bits\x18\x06 \x01(\x05\"\x93\x02\n\x18NaturalJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x91\x02\n\x16\x45xactJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x90\x02\n\x15LeftJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\xd1\x03\n\x15\x41sOfJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\\\n\x10\x61s_of_match_rule\x18\x07 \x01(\x0e\x32\x42.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.MatchRule\"]\n\tMatchRule\x12\x13\n\x0fLESS_THAN_EQUAL\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12GREATER_THAN_EQUAL\x10\x02\x12\x10\n\x0cGREATER_THAN\x10\x03\x1a\x02\x18\x01:\x02\x18\x01\"\xa6\x02\n\x12\x41jRajTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x14\n\x0c\x61s_of_column\x18\x05 \x01(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x06 \x03(\t\"\x88\x01\n\x0eMultiJoinInput\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x02 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x03 \x03(\t\"\xa4\x01\n\x16MultiJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x11multi_join_inputs\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.MultiJoinInput\"\xe0\x06\n\x16RangeJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x19\n\x11left_start_column\x18\x05 \x01(\t\x12\x62\n\x10range_start_rule\x18\x06 \x01(\x0e\x32H.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeStartRule\x12\x1a\n\x12right_range_column\x18\x07 \x01(\t\x12^\n\x0erange_end_rule\x18\x08 \x01(\x0e\x32\x46.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeEndRule\x12\x17\n\x0fleft_end_column\x18\t \x01(\t\x12\x44\n\x0c\x61ggregations\x18\n \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x13\n\x0brange_match\x18\x0b \x01(\t\"v\n\x0eRangeStartRule\x12\x15\n\x11START_UNSPECIFIED\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x02\x12&\n\"LESS_THAN_OR_EQUAL_ALLOW_PRECEDING\x10\x03\"{\n\x0cRangeEndRule\x12\x13\n\x0f\x45ND_UNSPECIFIED\x10\x00\x12\x10\n\x0cGREATER_THAN\x10\x01\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x02\x12)\n%GREATER_THAN_OR_EQUAL_ALLOW_FOLLOWING\x10\x03\"\xfe\x04\n\x15\x43omboAggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12V\n\naggregates\x18\x03 \x03(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\x12\x13\n\x0b\x66orce_combo\x18\x05 \x01(\x08\x1a\xad\x01\n\tAggregate\x12N\n\x04type\x18\x01 \x01(\x0e\x32@.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x12\n\npercentile\x18\x04 \x01(\x01\x12\x12\n\navg_median\x18\x05 \x01(\x08\"\xa5\x01\n\x07\x41ggType\x12\x07\n\x03SUM\x10\x00\x12\x0b\n\x07\x41\x42S_SUM\x10\x01\x12\t\n\x05GROUP\x10\x02\x12\x07\n\x03\x41VG\x10\x03\x12\t\n\x05\x43OUNT\x10\x04\x12\t\n\x05\x46IRST\x10\x05\x12\x08\n\x04LAST\x10\x06\x12\x07\n\x03MIN\x10\x07\x12\x07\n\x03MAX\x10\x08\x12\n\n\x06MEDIAN\x10\t\x12\x0e\n\nPERCENTILE\x10\n\x12\x07\n\x03STD\x10\x0b\x12\x07\n\x03VAR\x10\x0c\x12\x10\n\x0cWEIGHTED_AVG\x10\r:\x02\x18\x01\"\xed\x01\n\x13\x41ggregateAllRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x38\n\x04spec\x18\x03 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\"\xd7\x17\n\x07\x41ggSpec\x12K\n\x07\x61\x62s_sum\x18\x01 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSumH\x00\x12i\n\x16\x61pproximate_percentile\x18\x02 \x01(\x0b\x32G.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentileH\x00\x12\x44\n\x03\x61vg\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvgH\x00\x12Y\n\x0e\x63ount_distinct\x18\x04 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinctH\x00\x12N\n\x08\x64istinct\x18\x05 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinctH\x00\x12H\n\x05\x66irst\x18\x06 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirstH\x00\x12L\n\x07\x66ormula\x18\x07 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormulaH\x00\x12J\n\x06\x66reeze\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreezeH\x00\x12H\n\x05group\x18\t \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroupH\x00\x12\x46\n\x04last\x18\n \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLastH\x00\x12\x44\n\x03max\x18\x0b \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMaxH\x00\x12J\n\x06median\x18\x0c \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedianH\x00\x12\x44\n\x03min\x18\r \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMinH\x00\x12R\n\npercentile\x18\x0e \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentileH\x00\x12P\n\x0csorted_first\x18\x0f \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12O\n\x0bsorted_last\x18\x10 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12\x44\n\x03std\x18\x11 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStdH\x00\x12\x44\n\x03sum\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSumH\x00\x12M\n\x08t_digest\x18\x13 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigestH\x00\x12J\n\x06unique\x18\x14 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUniqueH\x00\x12R\n\x0cweighted_avg\x18\x15 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12R\n\x0cweighted_sum\x18\x16 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12\x44\n\x03var\x18\x17 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVarH\x00\x1a\\\n\x1c\x41ggSpecApproximatePercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x18\n\x0b\x63ompression\x18\x02 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a+\n\x14\x41ggSpecCountDistinct\x12\x13\n\x0b\x63ount_nulls\x18\x01 \x01(\x08\x1a(\n\x0f\x41ggSpecDistinct\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x1a\x36\n\x0e\x41ggSpecFormula\x12\x0f\n\x07\x66ormula\x18\x01 \x01(\t\x12\x13\n\x0bparam_token\x18\x02 \x01(\t\x1a/\n\rAggSpecMedian\x12\x1e\n\x16\x61verage_evenly_divided\x18\x01 \x01(\x08\x1aG\n\x11\x41ggSpecPercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x1e\n\x16\x61verage_evenly_divided\x18\x02 \x01(\x08\x1a`\n\rAggSpecSorted\x12O\n\x07\x63olumns\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn\x1a*\n\x13\x41ggSpecSortedColumn\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a:\n\x0e\x41ggSpecTDigest\x12\x18\n\x0b\x63ompression\x18\x01 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a\x88\x01\n\rAggSpecUnique\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x12`\n\x13non_unique_sentinel\x18\x02 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel\x1a\xb5\x02\n\x18\x41ggSpecNonUniqueSentinel\x12\x42\n\nnull_value\x18\x01 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.NullValueH\x00\x12\x16\n\x0cstring_value\x18\x02 \x01(\tH\x00\x12\x13\n\tint_value\x18\x03 \x01(\x11H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x15\n\x0b\x66loat_value\x18\x05 \x01(\x02H\x00\x12\x16\n\x0c\x64ouble_value\x18\x06 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x07 \x01(\x08H\x00\x12\x14\n\nbyte_value\x18\x08 \x01(\x11H\x00\x12\x15\n\x0bshort_value\x18\t \x01(\x11H\x00\x12\x14\n\nchar_value\x18\n \x01(\x11H\x00\x42\x06\n\x04type\x1a(\n\x0f\x41ggSpecWeighted\x12\x15\n\rweight_column\x18\x01 \x01(\t\x1a\x0f\n\rAggSpecAbsSum\x1a\x0c\n\nAggSpecAvg\x1a\x0e\n\x0c\x41ggSpecFirst\x1a\x0f\n\rAggSpecFreeze\x1a\x0e\n\x0c\x41ggSpecGroup\x1a\r\n\x0b\x41ggSpecLast\x1a\x0c\n\nAggSpecMax\x1a\x0c\n\nAggSpecMin\x1a\x0c\n\nAggSpecStd\x1a\x0c\n\nAggSpecSum\x1a\x0c\n\nAggSpecVarB\x06\n\x04type\"\xdc\x02\n\x10\x41ggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12L\n\x11initial_groups_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x16\n\x0epreserve_empty\x18\x04 \x01(\x08\x12\x44\n\x0c\x61ggregations\x18\x05 \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x18\n\x10group_by_columns\x18\x06 \x03(\t\"\x82\x07\n\x0b\x41ggregation\x12T\n\x07\x63olumns\x18\x01 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumnsH\x00\x12P\n\x05\x63ount\x18\x02 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountH\x00\x12Y\n\rfirst_row_key\x18\x03 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\x0clast_row_key\x18\x04 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\tpartition\x18\x05 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartitionH\x00\x12T\n\x07\x66ormula\x18\x06 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormulaH\x00\x1a\x63\n\x12\x41ggregationColumns\x12\x38\n\x04spec\x18\x01 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\'\n\x10\x41ggregationCount\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a(\n\x11\x41ggregationRowKey\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1aM\n\x14\x41ggregationPartition\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12 \n\x18include_group_by_columns\x18\x02 \x01(\x08\x1aW\n\x12\x41ggregationFormula\x12\x41\n\nselectable\x18\x01 \x01(\x0b\x32-.io.deephaven.proto.backplane.grpc.SelectableB\x06\n\x04type\"\xe1\x01\n\x0eSortDescriptor\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x13\n\x0bis_absolute\x18\x02 \x01(\x08\x12R\n\tdirection\x18\x03 \x01(\x0e\x32?.io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection\"Q\n\rSortDirection\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x17\n\nDESCENDING\x10\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01\x12\r\n\tASCENDING\x10\x01\x12\x0b\n\x07REVERSE\x10\x02\"\xd8\x01\n\x10SortTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12@\n\x05sorts\x18\x03 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.SortDescriptor\"\xd7\x01\n\x12\x46ilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12=\n\x07\x66ilters\x18\x03 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xf9\x01\n\x0eSeekRowRequest\x12<\n\tsource_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x18\n\x0cstarting_row\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12>\n\nseek_value\x18\x04 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.Literal\x12\x13\n\x0binsensitive\x18\x05 \x01(\x08\x12\x10\n\x08\x63ontains\x18\x06 \x01(\x08\x12\x13\n\x0bis_backward\x18\x07 \x01(\x08\")\n\x0fSeekRowResponse\x12\x16\n\nresult_row\x18\x01 \x01(\x12\x42\x02\x30\x01\" \n\tReference\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\"\x91\x01\n\x07Literal\x12\x16\n\x0cstring_value\x18\x01 \x01(\tH\x00\x12\x16\n\x0c\x64ouble_value\x18\x02 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x03 \x01(\x08H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1d\n\x0fnano_time_value\x18\x05 \x01(\x12\x42\x02\x30\x01H\x00\x42\x07\n\x05value\"\x91\x01\n\x05Value\x12\x41\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.ReferenceH\x00\x12=\n\x07literal\x18\x02 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.LiteralH\x00\x42\x06\n\x04\x64\x61ta\"\xbc\x05\n\tCondition\x12>\n\x03\x61nd\x18\x01 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.AndConditionH\x00\x12<\n\x02or\x18\x02 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.OrConditionH\x00\x12>\n\x03not\x18\x03 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.NotConditionH\x00\x12\x46\n\x07\x63ompare\x18\x04 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.CompareConditionH\x00\x12<\n\x02in\x18\x05 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.InConditionH\x00\x12\x44\n\x06invoke\x18\x06 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.InvokeConditionH\x00\x12\x45\n\x07is_null\x18\x07 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.IsNullConditionH\x00\x12\x46\n\x07matches\x18\x08 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MatchesConditionH\x00\x12H\n\x08\x63ontains\x18\t \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.ContainsConditionH\x00\x12\x44\n\x06search\x18\n \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.SearchConditionH\x00\x42\x06\n\x04\x64\x61ta\"M\n\x0c\x41ndCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0bOrCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0cNotCondition\x12<\n\x06\x66ilter\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xac\x03\n\x10\x43ompareCondition\x12W\n\toperation\x18\x01 \x01(\x0e\x32\x44.io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation\x12L\n\x10\x63\x61se_sensitivity\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12\x35\n\x03lhs\x18\x03 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12\x35\n\x03rhs\x18\x04 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"\x82\x01\n\x10\x43ompareOperation\x12\r\n\tLESS_THAN\x10\x00\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x01\x12\x10\n\x0cGREATER_THAN\x10\x02\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x03\x12\n\n\x06\x45QUALS\x10\x04\x12\x0e\n\nNOT_EQUALS\x10\x05\"\x95\x02\n\x0bInCondition\x12\x38\n\x06target\x18\x01 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12<\n\ncandidates\x18\x02 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\x98\x01\n\x0fInvokeCondition\x12\x0e\n\x06method\x18\x01 \x01(\t\x12\x38\n\x06target\x18\x02 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12;\n\targuments\x18\x03 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"R\n\x0fIsNullCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\xf2\x01\n\x10MatchesCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\r\n\x05regex\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\xfb\x01\n\x11\x43ontainsCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\x15\n\rsearch_string\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"s\n\x0fSearchCondition\x12\x15\n\rsearch_string\x18\x01 \x01(\t\x12I\n\x13optional_references\x18\x02 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\x94\x01\n\x0e\x46lattenRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\x96\x01\n\x10MetaTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb4\x03\n\x19RunChartDownsampleRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0bpixel_count\x18\x03 \x01(\x05\x12Z\n\nzoom_range\x18\x04 \x01(\x0b\x32\x46.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange\x12\x15\n\rx_column_name\x18\x05 \x01(\t\x12\x16\n\x0ey_column_names\x18\x06 \x03(\t\x1as\n\tZoomRange\x12\x1f\n\x0emin_date_nanos\x18\x01 \x01(\x03\x42\x02\x30\x01H\x00\x88\x01\x01\x12\x1f\n\x0emax_date_nanos\x18\x02 \x01(\x03\x42\x02\x30\x01H\x01\x88\x01\x01\x42\x11\n\x0f_min_date_nanosB\x11\n\x0f_max_date_nanos\"\xe0\x05\n\x17\x43reateInputTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x0fsource_table_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReferenceH\x00\x12\x10\n\x06schema\x18\x03 \x01(\x0cH\x00\x12W\n\x04kind\x18\x04 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind\x1a\xbf\x03\n\x0eInputTableKind\x12}\n\x15in_memory_append_only\x18\x01 \x01(\x0b\x32\\.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnlyH\x00\x12{\n\x14in_memory_key_backed\x18\x02 \x01(\x0b\x32[.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBackedH\x00\x12`\n\x05\x62link\x18\x03 \x01(\x0b\x32O.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.BlinkH\x00\x1a\x14\n\x12InMemoryAppendOnly\x1a(\n\x11InMemoryKeyBacked\x12\x13\n\x0bkey_columns\x18\x01 \x03(\t\x1a\x07\n\x05\x42linkB\x06\n\x04kindB\x0c\n\ndefinition\"\x83\x02\n\x0eWhereInRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x10\n\x08inverted\x18\x04 \x01(\x08\x12\x18\n\x10\x63olumns_to_match\x18\x05 \x03(\t\"\xea\x01\n\x17\x43olumnStatisticsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x1f\n\x12unique_value_limit\x18\x04 \x01(\x05H\x00\x88\x01\x01\x42\x15\n\x13_unique_value_limit\"\xdd\x01\n\x0cSliceRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12$\n\x18\x66irst_position_inclusive\x18\x03 \x01(\x12\x42\x02\x30\x01\x12#\n\x17last_position_exclusive\x18\x04 \x01(\x12\x42\x02\x30\x01\"\xdb\x1a\n\x11\x42\x61tchTableRequest\x12K\n\x03ops\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation\x1a\xf8\x19\n\tOperation\x12K\n\x0b\x65mpty_table\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequestH\x00\x12I\n\ntime_table\x18\x02 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.TimeTableRequestH\x00\x12M\n\x0c\x64rop_columns\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequestH\x00\x12J\n\x06update\x18\x04 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0blazy_update\x18\x05 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12H\n\x04view\x18\x06 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0bupdate_view\x18\x07 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12J\n\x06select\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12S\n\x0fselect_distinct\x18\t \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequestH\x00\x12G\n\x06\x66ilter\x18\n \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.FilterTableRequestH\x00\x12`\n\x13unstructured_filter\x18\x0b \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequestH\x00\x12\x43\n\x04sort\x18\x0c \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.SortTableRequestH\x00\x12\x44\n\x04head\x18\r \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12\x44\n\x04tail\x18\x0e \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12I\n\x07head_by\x18\x0f \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12I\n\x07tail_by\x18\x10 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12\x44\n\x07ungroup\x18\x11 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.UngroupRequestH\x00\x12\x46\n\x05merge\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequestH\x00\x12S\n\x0f\x63ombo_aggregate\x18\x13 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequestH\x00\x12\x44\n\x07\x66latten\x18\x15 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.FlattenRequestH\x00\x12\\\n\x14run_chart_downsample\x18\x16 \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequestH\x00\x12O\n\ncross_join\x18\x17 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequestH\x00\x12S\n\x0cnatural_join\x18\x18 \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequestH\x00\x12O\n\nexact_join\x18\x19 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequestH\x00\x12M\n\tleft_join\x18\x1a \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequestH\x00\x12R\n\nas_of_join\x18\x1b \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequestB\x02\x18\x01H\x00\x12K\n\x0b\x66\x65tch_table\x18\x1c \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.FetchTableRequestH\x00\x12^\n\x15\x61pply_preview_columns\x18\x1e \x01(\x0b\x32=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequestH\x00\x12X\n\x12\x63reate_input_table\x18\x1f \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.CreateInputTableRequestH\x00\x12G\n\tupdate_by\x18 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.UpdateByRequestH\x00\x12\x45\n\x08where_in\x18! \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.WhereInRequestH\x00\x12O\n\raggregate_all\x18\" \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequestH\x00\x12H\n\taggregate\x18# \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.AggregateRequestH\x00\x12K\n\x08snapshot\x18$ \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequestH\x00\x12T\n\rsnapshot_when\x18% \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequestH\x00\x12I\n\nmeta_table\x18& \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MetaTableRequestH\x00\x12O\n\nrange_join\x18\' \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequestH\x00\x12\x43\n\x02\x61j\x18( \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12\x44\n\x03raj\x18) \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12W\n\x11\x63olumn_statistics\x18* \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequestH\x00\x12O\n\nmulti_join\x18+ \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequestH\x00\x12@\n\x05slice\x18, \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.SliceRequestH\x00\x42\x04\n\x02opJ\x04\x08\x14\x10\x15J\x04\x08\x1d\x10\x1e*b\n\x0f\x42\x61\x64\x44\x61taBehavior\x12#\n\x1f\x42\x41\x44_DATA_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\t\n\x05THROW\x10\x01\x12\t\n\x05RESET\x10\x02\x12\x08\n\x04SKIP\x10\x03\x12\n\n\x06POISON\x10\x04*t\n\x14UpdateByNullBehavior\x12\x1f\n\x1bNULL_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\x12\n\x0eNULL_DOMINATES\x10\x01\x12\x13\n\x0fVALUE_DOMINATES\x10\x02\x12\x12\n\x0eZERO_DOMINATES\x10\x03*\x1b\n\tNullValue\x12\x0e\n\nNULL_VALUE\x10\x00*2\n\x0f\x43\x61seSensitivity\x12\x0e\n\nMATCH_CASE\x10\x00\x12\x0f\n\x0bIGNORE_CASE\x10\x01*&\n\tMatchType\x12\x0b\n\x07REGULAR\x10\x00\x12\x0c\n\x08INVERTED\x10\x01\x32\xb9\x32\n\x0cTableService\x12\x91\x01\n GetExportedTableCreationResponse\x12).io.deephaven.proto.backplane.grpc.Ticket\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nFetchTable\x12\x34.io.deephaven.proto.backplane.grpc.FetchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x98\x01\n\x13\x41pplyPreviewColumns\x12=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nEmptyTable\x12\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tTimeTable\x12\x33.io.deephaven.proto.backplane.grpc.TimeTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0b\x44ropColumns\x12\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Update\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nLazyUpdate\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x04View\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nUpdateView\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Select\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x82\x01\n\x08UpdateBy\x12\x32.io.deephaven.proto.backplane.grpc.UpdateByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eSelectDistinct\x12\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x06\x46ilter\x12\x35.io.deephaven.proto.backplane.grpc.FilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x9b\x01\n\x12UnstructuredFilter\x12\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x7f\n\x04Sort\x12\x33.io.deephaven.proto.backplane.grpc.SortTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Head\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Tail\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06HeadBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06TailBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07Ungroup\x12\x31.io.deephaven.proto.backplane.grpc.UngroupRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0bMergeTables\x12\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x43rossJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x94\x01\n\x11NaturalJoinTables\x12;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x45xactJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eLeftJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x41sOfJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x85\x01\n\x08\x41jTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\tRajTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fMultiJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fRangeJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x43omboAggregate\x12\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x8a\x01\n\x0c\x41ggregateAll\x12\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tAggregate\x12\x33.io.deephaven.proto.backplane.grpc.AggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x87\x01\n\x08Snapshot\x12\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8f\x01\n\x0cSnapshotWhen\x12;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07\x46latten\x12\x31.io.deephaven.proto.backplane.grpc.FlattenRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x96\x01\n\x12RunChartDownsample\x12<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x92\x01\n\x10\x43reateInputTable\x12:.io.deephaven.proto.backplane.grpc.CreateInputTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07WhereIn\x12\x31.io.deephaven.proto.backplane.grpc.WhereInRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x05\x42\x61tch\x12\x34.io.deephaven.proto.backplane.grpc.BatchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x30\x01\x12\x99\x01\n\x14\x45xportedTableUpdates\x12>.io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest\x1a=.io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage\"\x00\x30\x01\x12r\n\x07SeekRow\x12\x31.io.deephaven.proto.backplane.grpc.SeekRowRequest\x1a\x32.io.deephaven.proto.backplane.grpc.SeekRowResponse\"\x00\x12\x84\x01\n\tMetaTable\x12\x33.io.deephaven.proto.backplane.grpc.MetaTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x99\x01\n\x17\x43omputeColumnStatistics\x12:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12|\n\x05Slice\x12/.io.deephaven.proto.backplane.grpc.SliceRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x42\x41H\x01P\x01Z;github.com/deephaven/deephaven-core/go/internal/proto/tableb\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n deephaven_core/proto/table.proto\x12!io.deephaven.proto.backplane.grpc\x1a!deephaven_core/proto/ticket.proto\"l\n\x0eTableReference\x12;\n\x06ticket\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.TicketH\x00\x12\x16\n\x0c\x62\x61tch_offset\x18\x02 \x01(\x11H\x00\x42\x05\n\x03ref\"\xc6\x01\n\x1d\x45xportedTableCreationResponse\x12\x44\n\tresult_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07success\x18\x02 \x01(\x08\x12\x12\n\nerror_info\x18\x03 \x01(\t\x12\x15\n\rschema_header\x18\x04 \x01(\x0c\x12\x11\n\tis_static\x18\x05 \x01(\x08\x12\x10\n\x04size\x18\x06 \x01(\x12\x42\x02\x30\x01\"\x97\x01\n\x11\x46\x65tchTableRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\xa0\x01\n\x1a\x41pplyPreviewColumnsRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\x1d\n\x1b\x45xportedTableUpdatesRequest\"\x8c\x01\n\x1a\x45xportedTableUpdateMessage\x12<\n\texport_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x1e\n\x16update_failure_message\x18\x03 \x01(\t\"c\n\x11\x45mptyTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\"\xef\x01\n\x10TimeTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x1e\n\x10start_time_nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1b\n\x11start_time_string\x18\x05 \x01(\tH\x00\x12\x1a\n\x0cperiod_nanos\x18\x03 \x01(\x12\x42\x02\x30\x01H\x01\x12\x17\n\rperiod_string\x18\x06 \x01(\tH\x01\x12\x13\n\x0b\x62link_table\x18\x04 \x01(\x08\x42\x0c\n\nstart_timeB\x08\n\x06period\"\xb1\x01\n\x15SelectOrUpdateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_specs\x18\x03 \x03(\t\"#\n\nSelectable\x12\r\n\x03raw\x18\x01 \x01(\tH\x00\x42\x06\n\x04type\"\x8c\x02\n\x0bMathContext\x12\x11\n\tprecision\x18\x01 \x01(\x11\x12R\n\rrounding_mode\x18\x02 \x01(\x0e\x32;.io.deephaven.proto.backplane.grpc.MathContext.RoundingMode\"\x95\x01\n\x0cRoundingMode\x12\x1f\n\x1bROUNDING_MODE_NOT_SPECIFIED\x10\x00\x12\x06\n\x02UP\x10\x01\x12\x08\n\x04\x44OWN\x10\x02\x12\x0b\n\x07\x43\x45ILING\x10\x03\x12\t\n\x05\x46LOOR\x10\x04\x12\x0b\n\x07HALF_UP\x10\x05\x12\r\n\tHALF_DOWN\x10\x06\x12\r\n\tHALF_EVEN\x10\x07\x12\x0f\n\x0bUNNECESSARY\x10\x08\"\xdb\x02\n\x13UpdateByWindowScale\x12[\n\x05ticks\x18\x01 \x01(\x0b\x32J.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicksH\x00\x12Y\n\x04time\x18\x02 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTimeH\x00\x1a$\n\x13UpdateByWindowTicks\x12\r\n\x05ticks\x18\x01 \x01(\x01\x1a^\n\x12UpdateByWindowTime\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x13\n\x05nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x19\n\x0f\x64uration_string\x18\x03 \x01(\tH\x00\x42\x08\n\x06windowB\x06\n\x04type\"\xe1\x03\n\x11UpdateByEmOptions\x12I\n\ron_null_value\x18\x01 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_nan_value\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_null_time\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12R\n\x16on_negative_delta_time\x18\x04 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12N\n\x12on_zero_delta_time\x18\x05 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12I\n\x11\x62ig_value_context\x18\x06 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContext\"f\n\x14UpdateByDeltaOptions\x12N\n\rnull_behavior\x18\x01 \x01(\x0e\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByNullBehavior\"\x9b\x37\n\x0fUpdateByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12S\n\x07options\x18\x03 \x01(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions\x12X\n\noperations\x18\x04 \x03(\x0b\x32\x44.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation\x12\x18\n\x10group_by_columns\x18\x05 \x03(\t\x1a\xc3\x03\n\x0fUpdateByOptions\x12\x1c\n\x0fuse_redirection\x18\x01 \x01(\x08H\x00\x88\x01\x01\x12\x1b\n\x0e\x63hunk_capacity\x18\x02 \x01(\x05H\x01\x88\x01\x01\x12.\n!max_static_sparse_memory_overhead\x18\x03 \x01(\x01H\x02\x88\x01\x01\x12$\n\x17initial_hash_table_size\x18\x04 \x01(\x05H\x03\x88\x01\x01\x12 \n\x13maximum_load_factor\x18\x05 \x01(\x01H\x04\x88\x01\x01\x12\x1f\n\x12target_load_factor\x18\x06 \x01(\x01H\x05\x88\x01\x01\x12\x44\n\x0cmath_context\x18\x07 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContextB\x12\n\x10_use_redirectionB\x11\n\x0f_chunk_capacityB$\n\"_max_static_sparse_memory_overheadB\x1a\n\x18_initial_hash_table_sizeB\x16\n\x14_maximum_load_factorB\x15\n\x13_target_load_factor\x1a\xf4\x30\n\x11UpdateByOperation\x12\x65\n\x06\x63olumn\x18\x01 \x01(\x0b\x32S.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumnH\x00\x1a\xef/\n\x0eUpdateByColumn\x12n\n\x04spec\x18\x01 \x01(\x0b\x32`.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\xd7.\n\x0cUpdateBySpec\x12\x85\x01\n\x03sum\x18\x01 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSumH\x00\x12\x85\x01\n\x03min\x18\x02 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMinH\x00\x12\x85\x01\n\x03max\x18\x03 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMaxH\x00\x12\x8d\x01\n\x07product\x18\x04 \x01(\x0b\x32z.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProductH\x00\x12}\n\x04\x66ill\x18\x05 \x01(\x0b\x32m.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFillH\x00\x12{\n\x03\x65ma\x18\x06 \x01(\x0b\x32l.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmaH\x00\x12\x8a\x01\n\x0brolling_sum\x18\x07 \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSumH\x00\x12\x8e\x01\n\rrolling_group\x18\x08 \x01(\x0b\x32u.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroupH\x00\x12\x8a\x01\n\x0brolling_avg\x18\t \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvgH\x00\x12\x8a\x01\n\x0brolling_min\x18\n \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMinH\x00\x12\x8a\x01\n\x0brolling_max\x18\x0b \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMaxH\x00\x12\x92\x01\n\x0frolling_product\x18\x0c \x01(\x0b\x32w.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProductH\x00\x12\x7f\n\x05\x64\x65lta\x18\r \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDeltaH\x00\x12{\n\x03\x65ms\x18\x0e \x01(\x0b\x32l.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmsH\x00\x12\x80\x01\n\x06\x65m_min\x18\x0f \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMinH\x00\x12\x80\x01\n\x06\x65m_max\x18\x10 \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMaxH\x00\x12\x80\x01\n\x06\x65m_std\x18\x11 \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStdH\x00\x12\x8e\x01\n\rrolling_count\x18\x12 \x01(\x0b\x32u.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountH\x00\x12\x8a\x01\n\x0brolling_std\x18\x13 \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStdH\x00\x12\x8c\x01\n\x0crolling_wavg\x18\x14 \x01(\x0b\x32t.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvgH\x00\x12\x92\x01\n\x0frolling_formula\x18\x15 \x01(\x0b\x32w.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormulaH\x00\x1a\x17\n\x15UpdateByCumulativeSum\x1a\x17\n\x15UpdateByCumulativeMin\x1a\x17\n\x15UpdateByCumulativeMax\x1a\x1b\n\x19UpdateByCumulativeProduct\x1a\x0e\n\x0cUpdateByFill\x1a\xa2\x01\n\x0bUpdateByEma\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa2\x01\n\x0bUpdateByEms\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmMin\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmMax\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmStd\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1aY\n\rUpdateByDelta\x12H\n\x07options\x18\x01 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions\x1a\xc0\x01\n\x12UpdateByRollingSum\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc2\x01\n\x14UpdateByRollingGroup\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingAvg\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingMin\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingMax\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc4\x01\n\x16UpdateByRollingProduct\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc2\x01\n\x14UpdateByRollingCount\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingStd\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xd8\x01\n\x13UpdateByRollingWAvg\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12\x15\n\rweight_column\x18\x03 \x01(\t\x1a\xea\x01\n\x16UpdateByRollingFormula\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12\x0f\n\x07\x66ormula\x18\x03 \x01(\t\x12\x13\n\x0bparam_token\x18\x04 \x01(\tB\x06\n\x04typeB\x06\n\x04type\"\xb1\x01\n\x15SelectDistinctRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xae\x01\n\x12\x44ropColumnsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xb5\x01\n\x1eUnstructuredFilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07\x66ilters\x18\x03 \x03(\t\"\xad\x01\n\x11HeadOrTailRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\"\xce\x01\n\x13HeadOrTailByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\x12\x1d\n\x15group_by_column_specs\x18\x04 \x03(\t\"\xc3\x01\n\x0eUngroupRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x11\n\tnull_fill\x18\x03 \x01(\x08\x12\x1a\n\x12\x63olumns_to_ungroup\x18\x04 \x03(\t\"\xad\x01\n\x12MergeTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x45\n\nsource_ids\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x12\n\nkey_column\x18\x03 \x01(\t\"\x9a\x01\n\x14SnapshotTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb1\x02\n\x18SnapshotWhenTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07\x62\x61se_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x45\n\ntrigger_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07initial\x18\x04 \x01(\x08\x12\x13\n\x0bincremental\x18\x05 \x01(\x08\x12\x0f\n\x07history\x18\x06 \x01(\x08\x12\x15\n\rstamp_columns\x18\x07 \x03(\t\"\xa7\x02\n\x16\x43rossJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\x14\n\x0creserve_bits\x18\x06 \x01(\x05\"\x93\x02\n\x18NaturalJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x91\x02\n\x16\x45xactJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x90\x02\n\x15LeftJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\xd1\x03\n\x15\x41sOfJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\\\n\x10\x61s_of_match_rule\x18\x07 \x01(\x0e\x32\x42.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.MatchRule\"]\n\tMatchRule\x12\x13\n\x0fLESS_THAN_EQUAL\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12GREATER_THAN_EQUAL\x10\x02\x12\x10\n\x0cGREATER_THAN\x10\x03\x1a\x02\x18\x01:\x02\x18\x01\"\xa6\x02\n\x12\x41jRajTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x14\n\x0c\x61s_of_column\x18\x05 \x01(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x06 \x03(\t\"\x88\x01\n\x0eMultiJoinInput\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x02 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x03 \x03(\t\"\xa4\x01\n\x16MultiJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x11multi_join_inputs\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.MultiJoinInput\"\xe0\x06\n\x16RangeJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x19\n\x11left_start_column\x18\x05 \x01(\t\x12\x62\n\x10range_start_rule\x18\x06 \x01(\x0e\x32H.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeStartRule\x12\x1a\n\x12right_range_column\x18\x07 \x01(\t\x12^\n\x0erange_end_rule\x18\x08 \x01(\x0e\x32\x46.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeEndRule\x12\x17\n\x0fleft_end_column\x18\t \x01(\t\x12\x44\n\x0c\x61ggregations\x18\n \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x13\n\x0brange_match\x18\x0b \x01(\t\"v\n\x0eRangeStartRule\x12\x15\n\x11START_UNSPECIFIED\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x02\x12&\n\"LESS_THAN_OR_EQUAL_ALLOW_PRECEDING\x10\x03\"{\n\x0cRangeEndRule\x12\x13\n\x0f\x45ND_UNSPECIFIED\x10\x00\x12\x10\n\x0cGREATER_THAN\x10\x01\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x02\x12)\n%GREATER_THAN_OR_EQUAL_ALLOW_FOLLOWING\x10\x03\"\xfe\x04\n\x15\x43omboAggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12V\n\naggregates\x18\x03 \x03(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\x12\x13\n\x0b\x66orce_combo\x18\x05 \x01(\x08\x1a\xad\x01\n\tAggregate\x12N\n\x04type\x18\x01 \x01(\x0e\x32@.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x12\n\npercentile\x18\x04 \x01(\x01\x12\x12\n\navg_median\x18\x05 \x01(\x08\"\xa5\x01\n\x07\x41ggType\x12\x07\n\x03SUM\x10\x00\x12\x0b\n\x07\x41\x42S_SUM\x10\x01\x12\t\n\x05GROUP\x10\x02\x12\x07\n\x03\x41VG\x10\x03\x12\t\n\x05\x43OUNT\x10\x04\x12\t\n\x05\x46IRST\x10\x05\x12\x08\n\x04LAST\x10\x06\x12\x07\n\x03MIN\x10\x07\x12\x07\n\x03MAX\x10\x08\x12\n\n\x06MEDIAN\x10\t\x12\x0e\n\nPERCENTILE\x10\n\x12\x07\n\x03STD\x10\x0b\x12\x07\n\x03VAR\x10\x0c\x12\x10\n\x0cWEIGHTED_AVG\x10\r:\x02\x18\x01\"\xed\x01\n\x13\x41ggregateAllRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x38\n\x04spec\x18\x03 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\"\xd7\x17\n\x07\x41ggSpec\x12K\n\x07\x61\x62s_sum\x18\x01 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSumH\x00\x12i\n\x16\x61pproximate_percentile\x18\x02 \x01(\x0b\x32G.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentileH\x00\x12\x44\n\x03\x61vg\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvgH\x00\x12Y\n\x0e\x63ount_distinct\x18\x04 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinctH\x00\x12N\n\x08\x64istinct\x18\x05 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinctH\x00\x12H\n\x05\x66irst\x18\x06 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirstH\x00\x12L\n\x07\x66ormula\x18\x07 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormulaH\x00\x12J\n\x06\x66reeze\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreezeH\x00\x12H\n\x05group\x18\t \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroupH\x00\x12\x46\n\x04last\x18\n \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLastH\x00\x12\x44\n\x03max\x18\x0b \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMaxH\x00\x12J\n\x06median\x18\x0c \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedianH\x00\x12\x44\n\x03min\x18\r \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMinH\x00\x12R\n\npercentile\x18\x0e \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentileH\x00\x12P\n\x0csorted_first\x18\x0f \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12O\n\x0bsorted_last\x18\x10 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12\x44\n\x03std\x18\x11 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStdH\x00\x12\x44\n\x03sum\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSumH\x00\x12M\n\x08t_digest\x18\x13 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigestH\x00\x12J\n\x06unique\x18\x14 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUniqueH\x00\x12R\n\x0cweighted_avg\x18\x15 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12R\n\x0cweighted_sum\x18\x16 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12\x44\n\x03var\x18\x17 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVarH\x00\x1a\\\n\x1c\x41ggSpecApproximatePercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x18\n\x0b\x63ompression\x18\x02 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a+\n\x14\x41ggSpecCountDistinct\x12\x13\n\x0b\x63ount_nulls\x18\x01 \x01(\x08\x1a(\n\x0f\x41ggSpecDistinct\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x1a\x36\n\x0e\x41ggSpecFormula\x12\x0f\n\x07\x66ormula\x18\x01 \x01(\t\x12\x13\n\x0bparam_token\x18\x02 \x01(\t\x1a/\n\rAggSpecMedian\x12\x1e\n\x16\x61verage_evenly_divided\x18\x01 \x01(\x08\x1aG\n\x11\x41ggSpecPercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x1e\n\x16\x61verage_evenly_divided\x18\x02 \x01(\x08\x1a`\n\rAggSpecSorted\x12O\n\x07\x63olumns\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn\x1a*\n\x13\x41ggSpecSortedColumn\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a:\n\x0e\x41ggSpecTDigest\x12\x18\n\x0b\x63ompression\x18\x01 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a\x88\x01\n\rAggSpecUnique\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x12`\n\x13non_unique_sentinel\x18\x02 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel\x1a\xb5\x02\n\x18\x41ggSpecNonUniqueSentinel\x12\x42\n\nnull_value\x18\x01 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.NullValueH\x00\x12\x16\n\x0cstring_value\x18\x02 \x01(\tH\x00\x12\x13\n\tint_value\x18\x03 \x01(\x11H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x15\n\x0b\x66loat_value\x18\x05 \x01(\x02H\x00\x12\x16\n\x0c\x64ouble_value\x18\x06 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x07 \x01(\x08H\x00\x12\x14\n\nbyte_value\x18\x08 \x01(\x11H\x00\x12\x15\n\x0bshort_value\x18\t \x01(\x11H\x00\x12\x14\n\nchar_value\x18\n \x01(\x11H\x00\x42\x06\n\x04type\x1a(\n\x0f\x41ggSpecWeighted\x12\x15\n\rweight_column\x18\x01 \x01(\t\x1a\x0f\n\rAggSpecAbsSum\x1a\x0c\n\nAggSpecAvg\x1a\x0e\n\x0c\x41ggSpecFirst\x1a\x0f\n\rAggSpecFreeze\x1a\x0e\n\x0c\x41ggSpecGroup\x1a\r\n\x0b\x41ggSpecLast\x1a\x0c\n\nAggSpecMax\x1a\x0c\n\nAggSpecMin\x1a\x0c\n\nAggSpecStd\x1a\x0c\n\nAggSpecSum\x1a\x0c\n\nAggSpecVarB\x06\n\x04type\"\xdc\x02\n\x10\x41ggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12L\n\x11initial_groups_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x16\n\x0epreserve_empty\x18\x04 \x01(\x08\x12\x44\n\x0c\x61ggregations\x18\x05 \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x18\n\x10group_by_columns\x18\x06 \x03(\t\"\x9e\x08\n\x0b\x41ggregation\x12T\n\x07\x63olumns\x18\x01 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumnsH\x00\x12P\n\x05\x63ount\x18\x02 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountH\x00\x12Y\n\rfirst_row_key\x18\x03 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\x0clast_row_key\x18\x04 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\tpartition\x18\x05 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartitionH\x00\x12T\n\x07\x66ormula\x18\x06 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormulaH\x00\x12[\n\x0b\x63ount_where\x18\x07 \x01(\x0b\x32\x44.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhereH\x00\x1a\x63\n\x12\x41ggregationColumns\x12\x38\n\x04spec\x18\x01 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\'\n\x10\x41ggregationCount\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a=\n\x15\x41ggregationCountWhere\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x0f\n\x07\x66ilters\x18\x02 \x03(\t\x1a(\n\x11\x41ggregationRowKey\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1aM\n\x14\x41ggregationPartition\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12 \n\x18include_group_by_columns\x18\x02 \x01(\x08\x1aW\n\x12\x41ggregationFormula\x12\x41\n\nselectable\x18\x01 \x01(\x0b\x32-.io.deephaven.proto.backplane.grpc.SelectableB\x06\n\x04type\"\xe1\x01\n\x0eSortDescriptor\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x13\n\x0bis_absolute\x18\x02 \x01(\x08\x12R\n\tdirection\x18\x03 \x01(\x0e\x32?.io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection\"Q\n\rSortDirection\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x17\n\nDESCENDING\x10\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01\x12\r\n\tASCENDING\x10\x01\x12\x0b\n\x07REVERSE\x10\x02\"\xd8\x01\n\x10SortTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12@\n\x05sorts\x18\x03 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.SortDescriptor\"\xd7\x01\n\x12\x46ilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12=\n\x07\x66ilters\x18\x03 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xf9\x01\n\x0eSeekRowRequest\x12<\n\tsource_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x18\n\x0cstarting_row\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12>\n\nseek_value\x18\x04 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.Literal\x12\x13\n\x0binsensitive\x18\x05 \x01(\x08\x12\x10\n\x08\x63ontains\x18\x06 \x01(\x08\x12\x13\n\x0bis_backward\x18\x07 \x01(\x08\")\n\x0fSeekRowResponse\x12\x16\n\nresult_row\x18\x01 \x01(\x12\x42\x02\x30\x01\" \n\tReference\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\"\x91\x01\n\x07Literal\x12\x16\n\x0cstring_value\x18\x01 \x01(\tH\x00\x12\x16\n\x0c\x64ouble_value\x18\x02 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x03 \x01(\x08H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1d\n\x0fnano_time_value\x18\x05 \x01(\x12\x42\x02\x30\x01H\x00\x42\x07\n\x05value\"\x91\x01\n\x05Value\x12\x41\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.ReferenceH\x00\x12=\n\x07literal\x18\x02 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.LiteralH\x00\x42\x06\n\x04\x64\x61ta\"\xbc\x05\n\tCondition\x12>\n\x03\x61nd\x18\x01 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.AndConditionH\x00\x12<\n\x02or\x18\x02 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.OrConditionH\x00\x12>\n\x03not\x18\x03 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.NotConditionH\x00\x12\x46\n\x07\x63ompare\x18\x04 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.CompareConditionH\x00\x12<\n\x02in\x18\x05 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.InConditionH\x00\x12\x44\n\x06invoke\x18\x06 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.InvokeConditionH\x00\x12\x45\n\x07is_null\x18\x07 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.IsNullConditionH\x00\x12\x46\n\x07matches\x18\x08 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MatchesConditionH\x00\x12H\n\x08\x63ontains\x18\t \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.ContainsConditionH\x00\x12\x44\n\x06search\x18\n \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.SearchConditionH\x00\x42\x06\n\x04\x64\x61ta\"M\n\x0c\x41ndCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0bOrCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0cNotCondition\x12<\n\x06\x66ilter\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xac\x03\n\x10\x43ompareCondition\x12W\n\toperation\x18\x01 \x01(\x0e\x32\x44.io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation\x12L\n\x10\x63\x61se_sensitivity\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12\x35\n\x03lhs\x18\x03 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12\x35\n\x03rhs\x18\x04 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"\x82\x01\n\x10\x43ompareOperation\x12\r\n\tLESS_THAN\x10\x00\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x01\x12\x10\n\x0cGREATER_THAN\x10\x02\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x03\x12\n\n\x06\x45QUALS\x10\x04\x12\x0e\n\nNOT_EQUALS\x10\x05\"\x95\x02\n\x0bInCondition\x12\x38\n\x06target\x18\x01 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12<\n\ncandidates\x18\x02 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\x98\x01\n\x0fInvokeCondition\x12\x0e\n\x06method\x18\x01 \x01(\t\x12\x38\n\x06target\x18\x02 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12;\n\targuments\x18\x03 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"R\n\x0fIsNullCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\xf2\x01\n\x10MatchesCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\r\n\x05regex\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\xfb\x01\n\x11\x43ontainsCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\x15\n\rsearch_string\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"s\n\x0fSearchCondition\x12\x15\n\rsearch_string\x18\x01 \x01(\t\x12I\n\x13optional_references\x18\x02 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\x94\x01\n\x0e\x46lattenRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\x96\x01\n\x10MetaTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb4\x03\n\x19RunChartDownsampleRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0bpixel_count\x18\x03 \x01(\x05\x12Z\n\nzoom_range\x18\x04 \x01(\x0b\x32\x46.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange\x12\x15\n\rx_column_name\x18\x05 \x01(\t\x12\x16\n\x0ey_column_names\x18\x06 \x03(\t\x1as\n\tZoomRange\x12\x1f\n\x0emin_date_nanos\x18\x01 \x01(\x03\x42\x02\x30\x01H\x00\x88\x01\x01\x12\x1f\n\x0emax_date_nanos\x18\x02 \x01(\x03\x42\x02\x30\x01H\x01\x88\x01\x01\x42\x11\n\x0f_min_date_nanosB\x11\n\x0f_max_date_nanos\"\xe0\x05\n\x17\x43reateInputTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x0fsource_table_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReferenceH\x00\x12\x10\n\x06schema\x18\x03 \x01(\x0cH\x00\x12W\n\x04kind\x18\x04 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind\x1a\xbf\x03\n\x0eInputTableKind\x12}\n\x15in_memory_append_only\x18\x01 \x01(\x0b\x32\\.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnlyH\x00\x12{\n\x14in_memory_key_backed\x18\x02 \x01(\x0b\x32[.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBackedH\x00\x12`\n\x05\x62link\x18\x03 \x01(\x0b\x32O.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.BlinkH\x00\x1a\x14\n\x12InMemoryAppendOnly\x1a(\n\x11InMemoryKeyBacked\x12\x13\n\x0bkey_columns\x18\x01 \x03(\t\x1a\x07\n\x05\x42linkB\x06\n\x04kindB\x0c\n\ndefinition\"\x83\x02\n\x0eWhereInRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x10\n\x08inverted\x18\x04 \x01(\x08\x12\x18\n\x10\x63olumns_to_match\x18\x05 \x03(\t\"\xea\x01\n\x17\x43olumnStatisticsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x1f\n\x12unique_value_limit\x18\x04 \x01(\x05H\x00\x88\x01\x01\x42\x15\n\x13_unique_value_limit\"\xdd\x01\n\x0cSliceRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12$\n\x18\x66irst_position_inclusive\x18\x03 \x01(\x12\x42\x02\x30\x01\x12#\n\x17last_position_exclusive\x18\x04 \x01(\x12\x42\x02\x30\x01\"\xdb\x1a\n\x11\x42\x61tchTableRequest\x12K\n\x03ops\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation\x1a\xf8\x19\n\tOperation\x12K\n\x0b\x65mpty_table\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequestH\x00\x12I\n\ntime_table\x18\x02 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.TimeTableRequestH\x00\x12M\n\x0c\x64rop_columns\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequestH\x00\x12J\n\x06update\x18\x04 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0blazy_update\x18\x05 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12H\n\x04view\x18\x06 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0bupdate_view\x18\x07 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12J\n\x06select\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12S\n\x0fselect_distinct\x18\t \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequestH\x00\x12G\n\x06\x66ilter\x18\n \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.FilterTableRequestH\x00\x12`\n\x13unstructured_filter\x18\x0b \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequestH\x00\x12\x43\n\x04sort\x18\x0c \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.SortTableRequestH\x00\x12\x44\n\x04head\x18\r \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12\x44\n\x04tail\x18\x0e \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12I\n\x07head_by\x18\x0f \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12I\n\x07tail_by\x18\x10 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12\x44\n\x07ungroup\x18\x11 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.UngroupRequestH\x00\x12\x46\n\x05merge\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequestH\x00\x12S\n\x0f\x63ombo_aggregate\x18\x13 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequestH\x00\x12\x44\n\x07\x66latten\x18\x15 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.FlattenRequestH\x00\x12\\\n\x14run_chart_downsample\x18\x16 \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequestH\x00\x12O\n\ncross_join\x18\x17 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequestH\x00\x12S\n\x0cnatural_join\x18\x18 \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequestH\x00\x12O\n\nexact_join\x18\x19 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequestH\x00\x12M\n\tleft_join\x18\x1a \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequestH\x00\x12R\n\nas_of_join\x18\x1b \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequestB\x02\x18\x01H\x00\x12K\n\x0b\x66\x65tch_table\x18\x1c \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.FetchTableRequestH\x00\x12^\n\x15\x61pply_preview_columns\x18\x1e \x01(\x0b\x32=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequestH\x00\x12X\n\x12\x63reate_input_table\x18\x1f \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.CreateInputTableRequestH\x00\x12G\n\tupdate_by\x18 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.UpdateByRequestH\x00\x12\x45\n\x08where_in\x18! \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.WhereInRequestH\x00\x12O\n\raggregate_all\x18\" \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequestH\x00\x12H\n\taggregate\x18# \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.AggregateRequestH\x00\x12K\n\x08snapshot\x18$ \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequestH\x00\x12T\n\rsnapshot_when\x18% \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequestH\x00\x12I\n\nmeta_table\x18& \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MetaTableRequestH\x00\x12O\n\nrange_join\x18\' \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequestH\x00\x12\x43\n\x02\x61j\x18( \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12\x44\n\x03raj\x18) \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12W\n\x11\x63olumn_statistics\x18* \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequestH\x00\x12O\n\nmulti_join\x18+ \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequestH\x00\x12@\n\x05slice\x18, \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.SliceRequestH\x00\x42\x04\n\x02opJ\x04\x08\x14\x10\x15J\x04\x08\x1d\x10\x1e*b\n\x0f\x42\x61\x64\x44\x61taBehavior\x12#\n\x1f\x42\x41\x44_DATA_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\t\n\x05THROW\x10\x01\x12\t\n\x05RESET\x10\x02\x12\x08\n\x04SKIP\x10\x03\x12\n\n\x06POISON\x10\x04*t\n\x14UpdateByNullBehavior\x12\x1f\n\x1bNULL_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\x12\n\x0eNULL_DOMINATES\x10\x01\x12\x13\n\x0fVALUE_DOMINATES\x10\x02\x12\x12\n\x0eZERO_DOMINATES\x10\x03*\x1b\n\tNullValue\x12\x0e\n\nNULL_VALUE\x10\x00*2\n\x0f\x43\x61seSensitivity\x12\x0e\n\nMATCH_CASE\x10\x00\x12\x0f\n\x0bIGNORE_CASE\x10\x01*&\n\tMatchType\x12\x0b\n\x07REGULAR\x10\x00\x12\x0c\n\x08INVERTED\x10\x01\x32\xb9\x32\n\x0cTableService\x12\x91\x01\n GetExportedTableCreationResponse\x12).io.deephaven.proto.backplane.grpc.Ticket\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nFetchTable\x12\x34.io.deephaven.proto.backplane.grpc.FetchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x98\x01\n\x13\x41pplyPreviewColumns\x12=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nEmptyTable\x12\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tTimeTable\x12\x33.io.deephaven.proto.backplane.grpc.TimeTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0b\x44ropColumns\x12\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Update\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nLazyUpdate\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x04View\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nUpdateView\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Select\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x82\x01\n\x08UpdateBy\x12\x32.io.deephaven.proto.backplane.grpc.UpdateByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eSelectDistinct\x12\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x06\x46ilter\x12\x35.io.deephaven.proto.backplane.grpc.FilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x9b\x01\n\x12UnstructuredFilter\x12\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x7f\n\x04Sort\x12\x33.io.deephaven.proto.backplane.grpc.SortTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Head\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Tail\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06HeadBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06TailBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07Ungroup\x12\x31.io.deephaven.proto.backplane.grpc.UngroupRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0bMergeTables\x12\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x43rossJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x94\x01\n\x11NaturalJoinTables\x12;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x45xactJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eLeftJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x41sOfJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x85\x01\n\x08\x41jTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\tRajTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fMultiJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fRangeJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x43omboAggregate\x12\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x8a\x01\n\x0c\x41ggregateAll\x12\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tAggregate\x12\x33.io.deephaven.proto.backplane.grpc.AggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x87\x01\n\x08Snapshot\x12\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8f\x01\n\x0cSnapshotWhen\x12;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07\x46latten\x12\x31.io.deephaven.proto.backplane.grpc.FlattenRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x96\x01\n\x12RunChartDownsample\x12<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x92\x01\n\x10\x43reateInputTable\x12:.io.deephaven.proto.backplane.grpc.CreateInputTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07WhereIn\x12\x31.io.deephaven.proto.backplane.grpc.WhereInRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x05\x42\x61tch\x12\x34.io.deephaven.proto.backplane.grpc.BatchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x30\x01\x12\x99\x01\n\x14\x45xportedTableUpdates\x12>.io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest\x1a=.io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage\"\x00\x30\x01\x12r\n\x07SeekRow\x12\x31.io.deephaven.proto.backplane.grpc.SeekRowRequest\x1a\x32.io.deephaven.proto.backplane.grpc.SeekRowResponse\"\x00\x12\x84\x01\n\tMetaTable\x12\x33.io.deephaven.proto.backplane.grpc.MetaTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x99\x01\n\x17\x43omputeColumnStatistics\x12:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12|\n\x05Slice\x12/.io.deephaven.proto.backplane.grpc.SliceRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x42\x41H\x01P\x01Z;github.com/deephaven/deephaven-core/go/internal/proto/tableb\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -69,16 +69,16 @@ _globals['_TABLESERVICE'].methods_by_name['AsOfJoinTables']._serialized_options = b'\210\002\001' _globals['_TABLESERVICE'].methods_by_name['ComboAggregate']._loaded_options = None _globals['_TABLESERVICE'].methods_by_name['ComboAggregate']._serialized_options = b'\210\002\001' - _globals['_BADDATABEHAVIOR']._serialized_start=29165 - _globals['_BADDATABEHAVIOR']._serialized_end=29263 - _globals['_UPDATEBYNULLBEHAVIOR']._serialized_start=29265 - _globals['_UPDATEBYNULLBEHAVIOR']._serialized_end=29381 - _globals['_NULLVALUE']._serialized_start=29383 - _globals['_NULLVALUE']._serialized_end=29410 - _globals['_CASESENSITIVITY']._serialized_start=29412 - _globals['_CASESENSITIVITY']._serialized_end=29462 - _globals['_MATCHTYPE']._serialized_start=29464 - _globals['_MATCHTYPE']._serialized_end=29502 + _globals['_BADDATABEHAVIOR']._serialized_start=29321 + _globals['_BADDATABEHAVIOR']._serialized_end=29419 + _globals['_UPDATEBYNULLBEHAVIOR']._serialized_start=29421 + _globals['_UPDATEBYNULLBEHAVIOR']._serialized_end=29537 + _globals['_NULLVALUE']._serialized_start=29539 + _globals['_NULLVALUE']._serialized_end=29566 + _globals['_CASESENSITIVITY']._serialized_start=29568 + _globals['_CASESENSITIVITY']._serialized_end=29618 + _globals['_MATCHTYPE']._serialized_start=29620 + _globals['_MATCHTYPE']._serialized_end=29658 _globals['_TABLEREFERENCE']._serialized_start=106 _globals['_TABLEREFERENCE']._serialized_end=214 _globals['_EXPORTEDTABLECREATIONRESPONSE']._serialized_start=217 @@ -266,87 +266,89 @@ _globals['_AGGREGATEREQUEST']._serialized_start=18493 _globals['_AGGREGATEREQUEST']._serialized_end=18841 _globals['_AGGREGATION']._serialized_start=18844 - _globals['_AGGREGATION']._serialized_end=19742 - _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_start=19384 - _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_end=19483 - _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_start=19485 - _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_end=19524 - _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_start=19526 - _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_end=19566 - _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_start=19568 - _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_end=19645 - _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_start=19647 - _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_end=19734 - _globals['_SORTDESCRIPTOR']._serialized_start=19745 - _globals['_SORTDESCRIPTOR']._serialized_end=19970 - _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_start=19889 - _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_end=19970 - _globals['_SORTTABLEREQUEST']._serialized_start=19973 - _globals['_SORTTABLEREQUEST']._serialized_end=20189 - _globals['_FILTERTABLEREQUEST']._serialized_start=20192 - _globals['_FILTERTABLEREQUEST']._serialized_end=20407 - _globals['_SEEKROWREQUEST']._serialized_start=20410 - _globals['_SEEKROWREQUEST']._serialized_end=20659 - _globals['_SEEKROWRESPONSE']._serialized_start=20661 - _globals['_SEEKROWRESPONSE']._serialized_end=20702 - _globals['_REFERENCE']._serialized_start=20704 - _globals['_REFERENCE']._serialized_end=20736 - _globals['_LITERAL']._serialized_start=20739 - _globals['_LITERAL']._serialized_end=20884 - _globals['_VALUE']._serialized_start=20887 - _globals['_VALUE']._serialized_end=21032 - _globals['_CONDITION']._serialized_start=21035 - _globals['_CONDITION']._serialized_end=21735 - _globals['_ANDCONDITION']._serialized_start=21737 - _globals['_ANDCONDITION']._serialized_end=21814 - _globals['_ORCONDITION']._serialized_start=21816 - _globals['_ORCONDITION']._serialized_end=21892 - _globals['_NOTCONDITION']._serialized_start=21894 - _globals['_NOTCONDITION']._serialized_end=21970 - _globals['_COMPARECONDITION']._serialized_start=21973 - _globals['_COMPARECONDITION']._serialized_end=22401 - _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_start=22271 - _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_end=22401 - _globals['_INCONDITION']._serialized_start=22404 - _globals['_INCONDITION']._serialized_end=22681 - _globals['_INVOKECONDITION']._serialized_start=22684 - _globals['_INVOKECONDITION']._serialized_end=22836 - _globals['_ISNULLCONDITION']._serialized_start=22838 - _globals['_ISNULLCONDITION']._serialized_end=22920 - _globals['_MATCHESCONDITION']._serialized_start=22923 - _globals['_MATCHESCONDITION']._serialized_end=23165 - _globals['_CONTAINSCONDITION']._serialized_start=23168 - _globals['_CONTAINSCONDITION']._serialized_end=23419 - _globals['_SEARCHCONDITION']._serialized_start=23421 - _globals['_SEARCHCONDITION']._serialized_end=23536 - _globals['_FLATTENREQUEST']._serialized_start=23539 - _globals['_FLATTENREQUEST']._serialized_end=23687 - _globals['_METATABLEREQUEST']._serialized_start=23690 - _globals['_METATABLEREQUEST']._serialized_end=23840 - _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_start=23843 - _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_end=24279 - _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_start=24164 - _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_end=24279 - _globals['_CREATEINPUTTABLEREQUEST']._serialized_start=24282 - _globals['_CREATEINPUTTABLEREQUEST']._serialized_end=25018 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_start=24557 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_end=25004 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_start=24925 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_end=24945 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_start=24947 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_end=24987 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_start=24989 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_end=24996 - _globals['_WHEREINREQUEST']._serialized_start=25021 - _globals['_WHEREINREQUEST']._serialized_end=25280 - _globals['_COLUMNSTATISTICSREQUEST']._serialized_start=25283 - _globals['_COLUMNSTATISTICSREQUEST']._serialized_end=25517 - _globals['_SLICEREQUEST']._serialized_start=25520 - _globals['_SLICEREQUEST']._serialized_end=25741 - _globals['_BATCHTABLEREQUEST']._serialized_start=25744 - _globals['_BATCHTABLEREQUEST']._serialized_end=29163 - _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_start=25843 - _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_end=29163 - _globals['_TABLESERVICE']._serialized_start=29505 - _globals['_TABLESERVICE']._serialized_end=35962 + _globals['_AGGREGATION']._serialized_end=19898 + _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_start=19477 + _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_end=19576 + _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_start=19578 + _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_end=19617 + _globals['_AGGREGATION_AGGREGATIONCOUNTWHERE']._serialized_start=19619 + _globals['_AGGREGATION_AGGREGATIONCOUNTWHERE']._serialized_end=19680 + _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_start=19682 + _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_end=19722 + _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_start=19724 + _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_end=19801 + _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_start=19803 + _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_end=19890 + _globals['_SORTDESCRIPTOR']._serialized_start=19901 + _globals['_SORTDESCRIPTOR']._serialized_end=20126 + _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_start=20045 + _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_end=20126 + _globals['_SORTTABLEREQUEST']._serialized_start=20129 + _globals['_SORTTABLEREQUEST']._serialized_end=20345 + _globals['_FILTERTABLEREQUEST']._serialized_start=20348 + _globals['_FILTERTABLEREQUEST']._serialized_end=20563 + _globals['_SEEKROWREQUEST']._serialized_start=20566 + _globals['_SEEKROWREQUEST']._serialized_end=20815 + _globals['_SEEKROWRESPONSE']._serialized_start=20817 + _globals['_SEEKROWRESPONSE']._serialized_end=20858 + _globals['_REFERENCE']._serialized_start=20860 + _globals['_REFERENCE']._serialized_end=20892 + _globals['_LITERAL']._serialized_start=20895 + _globals['_LITERAL']._serialized_end=21040 + _globals['_VALUE']._serialized_start=21043 + _globals['_VALUE']._serialized_end=21188 + _globals['_CONDITION']._serialized_start=21191 + _globals['_CONDITION']._serialized_end=21891 + _globals['_ANDCONDITION']._serialized_start=21893 + _globals['_ANDCONDITION']._serialized_end=21970 + _globals['_ORCONDITION']._serialized_start=21972 + _globals['_ORCONDITION']._serialized_end=22048 + _globals['_NOTCONDITION']._serialized_start=22050 + _globals['_NOTCONDITION']._serialized_end=22126 + _globals['_COMPARECONDITION']._serialized_start=22129 + _globals['_COMPARECONDITION']._serialized_end=22557 + _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_start=22427 + _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_end=22557 + _globals['_INCONDITION']._serialized_start=22560 + _globals['_INCONDITION']._serialized_end=22837 + _globals['_INVOKECONDITION']._serialized_start=22840 + _globals['_INVOKECONDITION']._serialized_end=22992 + _globals['_ISNULLCONDITION']._serialized_start=22994 + _globals['_ISNULLCONDITION']._serialized_end=23076 + _globals['_MATCHESCONDITION']._serialized_start=23079 + _globals['_MATCHESCONDITION']._serialized_end=23321 + _globals['_CONTAINSCONDITION']._serialized_start=23324 + _globals['_CONTAINSCONDITION']._serialized_end=23575 + _globals['_SEARCHCONDITION']._serialized_start=23577 + _globals['_SEARCHCONDITION']._serialized_end=23692 + _globals['_FLATTENREQUEST']._serialized_start=23695 + _globals['_FLATTENREQUEST']._serialized_end=23843 + _globals['_METATABLEREQUEST']._serialized_start=23846 + _globals['_METATABLEREQUEST']._serialized_end=23996 + _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_start=23999 + _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_end=24435 + _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_start=24320 + _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_end=24435 + _globals['_CREATEINPUTTABLEREQUEST']._serialized_start=24438 + _globals['_CREATEINPUTTABLEREQUEST']._serialized_end=25174 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_start=24713 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_end=25160 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_start=25081 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_end=25101 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_start=25103 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_end=25143 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_start=25145 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_end=25152 + _globals['_WHEREINREQUEST']._serialized_start=25177 + _globals['_WHEREINREQUEST']._serialized_end=25436 + _globals['_COLUMNSTATISTICSREQUEST']._serialized_start=25439 + _globals['_COLUMNSTATISTICSREQUEST']._serialized_end=25673 + _globals['_SLICEREQUEST']._serialized_start=25676 + _globals['_SLICEREQUEST']._serialized_end=25897 + _globals['_BATCHTABLEREQUEST']._serialized_start=25900 + _globals['_BATCHTABLEREQUEST']._serialized_end=29319 + _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_start=25999 + _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_end=29319 + _globals['_TABLESERVICE']._serialized_start=29661 + _globals['_TABLESERVICE']._serialized_end=36118 # @@protoc_insertion_point(module_scope) diff --git a/py/client/pydeephaven/agg.py b/py/client/pydeephaven/agg.py index 83d9b1f40ac..de937e4be80 100644 --- a/py/client/pydeephaven/agg.py +++ b/py/client/pydeephaven/agg.py @@ -16,6 +16,7 @@ _GrpcAggregation = table_pb2.Aggregation _GrpcAggregationColumns = _GrpcAggregation.AggregationColumns _GrpcAggregationCount = _GrpcAggregation.AggregationCount +_GrpcAggregationCountWhere = _GrpcAggregation.AggregationCountWhere _GrpcAggregationFormula = _GrpcAggregation.AggregationFormula _GrpcAggregationPartition = _GrpcAggregation.AggregationPartition _GrpcAggSpec = table_pb2.AggSpec @@ -51,6 +52,17 @@ def make_grpc_message(self) -> _GrpcAggregation: agg_count = _GrpcAggregationCount(column_name=self.col) return _GrpcAggregation(count=agg_count) + +@dataclass +class _AggregationCountWhere(Aggregation): + col: str + filters: Union[str, List[str]] + + def make_grpc_message(self) -> _GrpcAggregation: + agg_count_where = _GrpcAggregationCountWhere(column_name=self.col, filters=to_list(self.filters)) + return _GrpcAggregation(count_where=agg_count_where) + + @dataclass class _AggregationFormula(Aggregation): selectable: _GrpcSelectable @@ -59,6 +71,7 @@ def make_grpc_message(self) -> _GrpcAggregation: agg_formula = _GrpcAggregationFormula(selectable=self.selectable) return _GrpcAggregation(formula=agg_formula) + @dataclass class _AggregationPartition(Aggregation): col: str @@ -137,6 +150,18 @@ def count_(col: str) -> Aggregation: return _AggregationCount(col=col) +def count_where(col: str, filters: Union[str, List[str]]) -> Aggregation: + """Creates a Count aggregation. This is not supported in 'Table.agg_all_by'. + + Args: + col (str): the column to hold the counts of each distinct group + + Returns: + an aggregation + """ + return _AggregationCountWhere(col=col, filters=to_list(filters)) + + def partition(col: str, include_by_columns: bool = True) -> Aggregation: """Creates a Partition aggregation. This is not supported in 'Table.agg_all_by'. diff --git a/py/client/tests/test_table.py b/py/client/tests/test_table.py index 25d4067a87a..e5db7717ba9 100644 --- a/py/client/tests/test_table.py +++ b/py/client/tests/test_table.py @@ -10,7 +10,7 @@ from pydeephaven import DHError from pydeephaven import SortDirection -from pydeephaven.agg import sum_, avg, pct, weighted_avg, count_, partition, median, unique, count_distinct, distinct, formula +from pydeephaven.agg import sum_, avg, pct, weighted_avg, count_, count_where, partition, median, unique, count_distinct, distinct, formula from pydeephaven.table import Table from tests.testbase import BaseTestCase @@ -247,6 +247,10 @@ def test_agg_by(self): pct(percentile=0.5, cols=["PctC = c"]), weighted_avg(wcol="d", cols=["WavGD = d"]), count_(col="ca"), + count_where(col="count_where1", filters="a > 5"), + count_where("agg_count_where_1", "a > 100"), + count_where("agg_count_where_2", ["a > 100", "b < 250"]), + count_where("agg_count_where_3", "a <= 100 || b >= 250"), partition(col="aggPartition"), formula(formula="min(x)", formula_param="x", cols=["min_a=a", "min_b=b"]), formula(formula="avg(x)", formula_param="x", cols=["avg_c=c", "avg_d=d"]), @@ -301,6 +305,26 @@ def test_agg_all_by(self): with self.assertRaises(DHError) as cm: test_table.agg_all_by(agg=count_(col="ca"), by=["a"]) + def test_agg_count_where_output(self): + """ + Test and validation of the agg_count_where feature + """ + test_table = self.session.empty_table(100).update(["a=ii", "b=ii%2"]) + count_aggs = [ + count_where("count1", "a >= 25"), + count_where("count2", "a % 3 == 0") + ] + result_table = test_table.agg_by(aggs=count_aggs, by="b") + self.assertEqual(result_table.size, 2) + + # get the table as a local pandas dataframe + df = result_table.to_arrow().to_pandas() + # assert the values meet expectations + self.assertEqual(df.loc[0, "count1"], 37) + self.assertEqual(df.loc[1, "count1"], 38) + self.assertEqual(df.loc[0, "count2"], 17) + self.assertEqual(df.loc[1, "count2"], 17) + def test_where_in(self): pa_table = csv.read_csv(self.csv_file) test_table = self.session.import_table(pa_table) diff --git a/py/server/deephaven/agg.py b/py/server/deephaven/agg.py index 222329f7803..45f7c4668ab 100644 --- a/py/server/deephaven/agg.py +++ b/py/server/deephaven/agg.py @@ -3,12 +3,13 @@ # """This module implement various aggregations that can be used in deephaven table's aggregation operations.""" -from typing import List, Union, Any, Optional +from typing import List, Union, Any, Optional, Sequence import jpy from deephaven import DHError from deephaven.jcompat import to_sequence +from deephaven.filters import Filter, and_ _JAggregation = jpy.get_type("io.deephaven.api.agg.Aggregation") _JAggSpec = jpy.get_type("io.deephaven.api.agg.spec.AggSpec") @@ -113,6 +114,22 @@ def count_(col: str) -> Aggregation: raise DHError(message="count_ aggregation requires a string value for the 'col' argument.") return Aggregation(j_aggregation=_JAggregation.AggCount(col)) +def count_where(col: str, filters: Union[str, Filter, Sequence[str], Sequence[Filter]]) -> Aggregation: + """Creates a CountWhere aggregation with the supplied output column name, counting values that pass the supplied + filters. + + Args: + col (str): the column to hold the counts of each distinct group + + Returns: + an aggregation + """ + if not isinstance(col, str): + raise DHError(message="count_where aggregation requires a string value for the 'col' argument.") + filters = to_sequence(filters) + + return Aggregation(j_aggregation=_JAggregation.AggCountWhere(col, and_(filters).j_filter)) + def partition(col: str, include_by_columns: bool = True) -> Aggregation: """Creates a Partition aggregation. This is not supported in 'Table.agg_all_by'. diff --git a/py/server/tests/test_table.py b/py/server/tests/test_table.py index 70854c5d90a..2315b6c842f 100644 --- a/py/server/tests/test_table.py +++ b/py/server/tests/test_table.py @@ -8,13 +8,14 @@ from deephaven import DHError, read_csv, empty_table, SortDirection, time_table, update_graph, new_table, dtypes from deephaven.agg import sum_, weighted_avg, avg, pct, group, count_, first, last, max_, median, min_, std, abs_sum, \ - var, formula, partition, unique, count_distinct, distinct + var, formula, partition, unique, count_distinct, distinct, count_where from deephaven.column import datetime_col from deephaven.execution_context import make_user_exec_ctx, get_exec_ctx from deephaven.html import to_html from deephaven.jcompat import j_hashmap from deephaven.pandas import to_pandas from deephaven.table import Table, TableDefinition, SearchDisplayMode, table_diff +from deephaven.filters import Filter, and_, or_ from tests.testbase import BaseTestCase, table_equals @@ -43,6 +44,9 @@ def setUp(self): self.aggs_for_rollup = [ avg(["aggAvg=var"]), count_("aggCount"), + count_where("aggCountWhere1", "var > 100"), + count_where("aggCountWhere2", ["var > 100", "var < 250"]), + count_where("aggCountWhere3", or_(["var > 100", "var < 250"])), first(["aggFirst=var"]), last(["aggLast=var"]), max_(["aggMax=var"]), @@ -483,6 +487,26 @@ def test_agg_by_2(self): result_table = test_table.agg_by(agg, "grp_id") self.assertEqual(result_table.size, 2) + def test_agg_count_where_output(self): + """ + Test and validation of the agg_count_where feature + """ + test_table = empty_table(100).update(["a=ii", "b=ii%2"]) + count_aggs = [ + count_where("count1", "a >= 25"), + count_where("count2", "a % 3 == 0") + ] + result_table = test_table.agg_by(aggs=count_aggs, by="b") + self.assertEqual(result_table.size, 2) + + # get the table as a local pandas dataframe + df = to_pandas(result_table) + # assert the values meet expectations + self.assertEqual(df.loc[0, "count1"], 37) + self.assertEqual(df.loc[1, "count1"], 38) + self.assertEqual(df.loc[0, "count2"], 17) + self.assertEqual(df.loc[1, "count2"], 17) + def test_agg_by_initial_groups_preserve_empty(self): test_table = empty_table(10) test_table = test_table.update( diff --git a/replication/static/src/main/java/io/deephaven/replicators/ReplicateChunkFilters.java b/replication/static/src/main/java/io/deephaven/replicators/ReplicateChunkFilters.java index 4c1c72974cd..3dc7c9dea24 100644 --- a/replication/static/src/main/java/io/deephaven/replicators/ReplicateChunkFilters.java +++ b/replication/static/src/main/java/io/deephaven/replicators/ReplicateChunkFilters.java @@ -12,11 +12,16 @@ public class ReplicateChunkFilters { private static final String TASK = "replicateChunkFilters"; public static void main(String[] args) throws IOException { + charToAllButBoolean(TASK, + "engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharChunkFilter.java"); + charToShortAndByte(TASK, "engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java"); charToInteger(TASK, "engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java", Collections.emptyMap()); + charToLong(TASK, + "engine/table/src/main/java/io/deephaven/engine/table/impl/chunkfilter/CharRangeComparator.java"); charToShortAndByte(TASK, "engine/table/src/main/java/io/deephaven/engine/table/impl/select/CharRangeFilter.java"); diff --git a/server/src/main/java/io/deephaven/server/table/ops/AggregationAdapter.java b/server/src/main/java/io/deephaven/server/table/ops/AggregationAdapter.java index 00eff2b9754..32d1ed448a1 100644 --- a/server/src/main/java/io/deephaven/server/table/ops/AggregationAdapter.java +++ b/server/src/main/java/io/deephaven/server/table/ops/AggregationAdapter.java @@ -6,18 +6,11 @@ import com.google.protobuf.Descriptors.FieldDescriptor; import com.google.protobuf.Message; import com.google.rpc.Code; -import io.deephaven.api.agg.Aggregation; -import io.deephaven.api.agg.Aggregations; -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.Formula; -import io.deephaven.api.agg.LastRowKey; -import io.deephaven.api.agg.Partition; +import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount; +import io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey; @@ -92,6 +85,10 @@ public static Count adapt(AggregationCount count) { return Aggregation.AggCount(count.getColumnName()); } + public static CountWhere adapt(AggregationCountWhere count) { + return Aggregation.AggCountWhere(count.getColumnName(), count.getFiltersList().toArray(String[]::new)); + } + public static Formula adapt(AggregationFormula formula) { return Formula.of(adapt(formula.getSelectable())); } @@ -181,6 +178,15 @@ public void visit(Count count) { AggregationAdapter::adapt); } + public void visit(CountWhere countWhere) { + add( + TypeCase.COUNT_WHERE, + AggregationCountWhere.class, + CountWhere.class, + GrpcErrorHelper::checkHasNoUnknownFieldsRecursive, + AggregationAdapter::adapt); + } + @Override public void visit(FirstRowKey firstRowKey) { add( 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 66050567636..b27f487aecf 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 @@ -17,6 +17,7 @@ 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.filter.Filter; import io.deephaven.api.object.UnionObject; import java.util.Arrays; @@ -224,6 +225,28 @@ static Count AggCount(String resultColumn) { return Count.of(resultColumn); } + /** + * Create a {@link io.deephaven.api.agg.CountWhere count} aggregation with the supplied output column name, counting + * values that pass the supplied {@code filters}. + * + * @param resultColumn The {@link Count#column() output column} name + * @return The aggregation + */ + static CountWhere AggCountWhere(String resultColumn, String... filters) { + return CountWhere.of(resultColumn, filters); + } + + /** + * Create a {@link io.deephaven.api.agg.CountWhere count} aggregation with the supplied output column name, counting + * values that pass the supplied {@code filter}. + * + * @param resultColumn The {@link Count#column() output column} name + * @return The aggregation + */ + static CountWhere AggCountWhere(String resultColumn, Filter filter) { + return CountWhere.of(resultColumn, filter); + } + /** * 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). @@ -559,7 +582,7 @@ static Aggregation AggSortedLast(Collection sortColumns, Strin /** * Create a {@link io.deephaven.api.agg.spec.AggSpecStd sample standard deviation} aggregation for the supplied * column name pairs. - * + *

    * Sample standard deviation is computed using Bessel's correction * (https://en.wikipedia.org/wiki/Bessel%27s_correction), which ensures that the sample variance will be an unbiased * estimator of population variance. @@ -649,7 +672,7 @@ static Aggregation AggUnique(boolean includeNulls, UnionObject nonUniqueSentinel /** * Create a {@link io.deephaven.api.agg.spec.AggSpecVar sample variance} aggregation for the supplied column name * pairs. - * + *

    * Sample variance is computed using Bessel's correction (https://en.wikipedia.org/wiki/Bessel%27s_correction), * which ensures that the sample variance will be an unbiased estimator of population variance. * @@ -710,6 +733,7 @@ static void visitAll(Visitor visitor) { visitor.visit((ColumnAggregation) null); visitor.visit((ColumnAggregations) null); visitor.visit((Count) null); + visitor.visit((CountWhere) null); visitor.visit((FirstRowKey) null); visitor.visit((LastRowKey) null); visitor.visit((Partition) null); @@ -758,6 +782,13 @@ interface Visitor { */ void visit(Count count); + /** + * Visit a {@link CountWhere count aggregation}. + * + * @param countWhere The count aggregation + */ + void visit(CountWhere countWhere); + /** * Visit a {@link FirstRowKey first row key aggregation}. * 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 ad435397fea..3a14a51c220 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 @@ -65,6 +65,11 @@ public void visit(Count count) { out.put(count.column().name(), "count"); } + @Override + public void visit(CountWhere countWhere) { + out.put(countWhere.column().name(), "countWhere " + Strings.of(countWhere.filter())); + } + @Override public void visit(FirstRowKey firstRowKey) { out.put(firstRowKey.column().name(), "first 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 b038de3217d..2c854541777 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 @@ -96,6 +96,12 @@ public void visit(Count count) { visitOrder.computeIfAbsent(COUNT_OBJ, k -> new ArrayList<>()).add(count.column()); } + @Override + public void visit(CountWhere countWhere) { + // Supplying a `null` entry value indicates that the key is already an aggregation. + visitOrder.putIfAbsent(countWhere, null); + } + @Override public void visit(FirstRowKey firstRowKey) { visitOrder.computeIfAbsent(FIRST_ROW_KEY_OBJ, k -> new ArrayList<>()).add(firstRowKey.column()); 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 f40acae6163..1715470fae9 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 @@ -49,6 +49,11 @@ public void visit(Count count) { out = Stream.of(count.column()); } + @Override + public void visit(CountWhere countWhere) { + out = Stream.of(countWhere.column()); + } + @Override public void visit(FirstRowKey firstRowKey) { out = Stream.of(firstRowKey.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 2087e4894af..269517037a4 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 @@ -49,6 +49,11 @@ public void visit(Count count) { out = Stream.empty(); } + @Override + public void visit(CountWhere countWhere) { + out = Stream.empty(); + } + @Override public void visit(FirstRowKey firstRowKey) { out = Stream.empty(); diff --git a/table-api/src/main/java/io/deephaven/api/agg/CountWhere.java b/table-api/src/main/java/io/deephaven/api/agg/CountWhere.java new file mode 100644 index 00000000000..9fe0f6b396e --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/CountWhere.java @@ -0,0 +1,45 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.api.agg; + +import io.deephaven.annotations.SimpleStyle; +import io.deephaven.api.ColumnName; +import io.deephaven.api.filter.Filter; +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 CountWhere implements Aggregation { + + public static CountWhere of(String name, String... filters) { + return of(ColumnName.of(name), Filter.and(Filter.from(filters))); + } + + public static CountWhere of(String name, Filter filter) { + return of(ColumnName.of(name), filter); + } + + public static CountWhere of(ColumnName name, Filter filter) { + return ImmutableCountWhere.of(name, filter); + } + + @Parameter + public abstract ColumnName column(); + + @Parameter + public abstract Filter filter(); + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } +} diff --git a/table-api/src/test/java/io/deephaven/api/agg/AggregationTest.java b/table-api/src/test/java/io/deephaven/api/agg/AggregationTest.java index ad9a0f3a3bc..f8f04931fd0 100644 --- a/table-api/src/test/java/io/deephaven/api/agg/AggregationTest.java +++ b/table-api/src/test/java/io/deephaven/api/agg/AggregationTest.java @@ -56,6 +56,11 @@ public void visit(Count c) { ++count; } + @Override + public void visit(CountWhere countWhere) { + ++count; + } + @Override public void visit(FirstRowKey firstRowKey) { ++count;