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 8ca700c7eda..3bf51eb5e34 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/AggregationProcessor.java @@ -3,9 +3,7 @@ // package io.deephaven.engine.table.impl.by; -import io.deephaven.api.ColumnName; -import io.deephaven.api.Pair; -import io.deephaven.api.SortColumn; +import io.deephaven.api.*; import io.deephaven.api.agg.*; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.agg.spec.AggSpecAbsSum; @@ -93,6 +91,7 @@ import io.deephaven.engine.table.impl.by.ssmcountdistinct.unique.ShortRollupUniqueOperator; import io.deephaven.engine.table.impl.by.ssmminmax.SsmChunkedMinMaxOperator; import io.deephaven.engine.table.impl.by.ssmpercentile.SsmChunkedPercentileOperator; +import io.deephaven.engine.table.impl.select.SelectColumn; 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; @@ -100,6 +99,7 @@ import io.deephaven.time.DateTimeUtils; import io.deephaven.util.annotations.FinalDefault; import io.deephaven.util.type.ArrayTypeUtils; +import io.deephaven.vector.VectorFactory; import org.apache.commons.lang3.mutable.MutableBoolean; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -113,6 +113,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; @@ -153,6 +154,13 @@ private enum Type { private final Collection aggregations; private final Type type; + /** + * For {@link Formula formula} aggregations we need a representation of the table definition with the column data + * types converted to {@link io.deephaven.vector.Vector vectors}. This can be computed once and re-used across all + * formula aggregations. + */ + private Map> vectorColumnDefinitions; + /** * Convert a collection of {@link Aggregation aggregations} to an {@link AggregationContextFactory}. * @@ -707,6 +715,50 @@ public void visit(@NotNull final Partition partition) { groupByColumnNames)); } + @Override + public void visit(@NotNull final Formula formula) { + final SelectColumn selectColumn = SelectColumn.of(formula.selectable()); + + // Get or create a column definition map composed of vectors of the original column types (or scalars when + // part of the key columns). + final Set groupByColumnSet = Set.of(groupByColumnNames); + if (vectorColumnDefinitions == null) { + vectorColumnDefinitions = table.getDefinition().getColumnStream().collect(Collectors.toMap( + ColumnDefinition::getName, + (final ColumnDefinition cd) -> groupByColumnSet.contains(cd.getName()) + ? cd + : ColumnDefinition.fromGenericType( + cd.getName(), + VectorFactory.forElementType(cd.getDataType()).vectorType(), + cd.getDataType()))); + } + + // Get the input column names from the formula and provide them to the groupBy operator + final String[] allInputColumns = + selectColumn.initDef(vectorColumnDefinitions, compilationProcessor).toArray(String[]::new); + + final Map> partitioned = Arrays.stream(allInputColumns) + .collect(Collectors.partitioningBy(groupByColumnSet::contains)); + final String[] inputKeyColumns = partitioned.get(true).toArray(String[]::new); + final String[] inputNonKeyColumns = partitioned.get(false).toArray(String[]::new); + + if (!selectColumn.getColumnArrays().isEmpty()) { + throw new IllegalArgumentException("AggFormula does not support column arrays (" + + selectColumn.getColumnArrays() + ")"); + } + if (selectColumn.hasVirtualRowVariables()) { + throw new IllegalArgumentException("AggFormula does not support virtual row variables"); + } + // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) + final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, null, + Arrays.stream(inputNonKeyColumns).map(col -> MatchPair.of(Pair.parse(col))) + .toArray(MatchPair[]::new)); + + final FormulaMultiColumnChunkedOperator op = new FormulaMultiColumnChunkedOperator(table, + groupByChunkedOperator, true, selectColumn, inputKeyColumns); + addNoInputOperator(op); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor // ------------------------------------------------------------------------------------------------------------- @@ -745,6 +797,7 @@ public void visit(@NotNull final AggSpecFirst first) { @Override public void visit(@NotNull final AggSpecFormula formula) { unsupportedForBlinkTables("Formula"); + // TODO: re-use shared groupBy operators (https://github.com/deephaven/deephaven-core/issues/6363) final GroupByChunkedOperator groupByChunkedOperator = new GroupByChunkedOperator(table, false, null, resultPairs.stream().map(pair -> MatchPair.of((Pair) pair.input())).toArray(MatchPair[]::new)); final FormulaChunkedOperator formulaChunkedOperator = new FormulaChunkedOperator(groupByChunkedOperator, @@ -860,6 +913,12 @@ default void visit(@NotNull final LastRowKey lastRowKey) { rollupUnsupported("LastRowKey"); } + @Override + @FinalDefault + default void visit(@NotNull final Formula formula) { + rollupUnsupported("Formula"); + } + // ------------------------------------------------------------------------------------------------------------- // AggSpec.Visitor for unsupported column aggregation specs // ------------------------------------------------------------------------------------------------------------- diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java index bdf69ed3a97..b0074d68ddc 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaChunkedOperator.java @@ -291,9 +291,8 @@ public UnaryOperator initializeRefreshing(@NotNull final Quer resultColumnModifiedColumnSets[ci] = resultTable.newModifiedColumnSet(resultColumnNames[ci]); } if (delegateToBy) { - // We cannot use the groupBy's result MCS factory, because the result column names are not guaranteed to be - // the - // same. + // We cannot use the groupBy's result MCS factory, because the result column names are not guaranteed + // to be the same. groupBy.initializeRefreshing(resultTable, aggregationUpdateListener); } // Note that we also use the factory in propagateUpdates to identify the set of modified columns to handle. @@ -379,7 +378,7 @@ private class DataFillerContext implements SafeCloseable { private final boolean[] columnsToFillMask; final FillFromContext[] fillFromContexts; - private DataFillerContext(@NotNull final boolean[] columnsToFillMask) { + private DataFillerContext(final boolean @NotNull [] columnsToFillMask) { this.columnsToFillMask = columnsToFillMask; fillFromContexts = new FillFromContext[resultColumnNames.length]; for (int ci = 0; ci < resultColumnNames.length; ++ci) { @@ -448,6 +447,7 @@ private void copyData(@NotNull final RowSequence rowSequence, @NotNull final boo rowSequenceSlice); } } + sharedContext.reset(); } } } diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java new file mode 100644 index 00000000000..0dd080c92e9 --- /dev/null +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/FormulaMultiColumnChunkedOperator.java @@ -0,0 +1,411 @@ +// +// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending +// +package io.deephaven.engine.table.impl.by; + +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.liveness.LivenessReferent; +import io.deephaven.engine.rowset.RowSequence; +import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.chunkattributes.RowKeys; +import io.deephaven.engine.table.ChunkSink.FillFromContext; +import io.deephaven.engine.table.*; +import io.deephaven.engine.table.ChunkSource.GetContext; +import io.deephaven.engine.table.impl.QueryTable; +import io.deephaven.engine.table.impl.select.SelectColumn; +import io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource; +import io.deephaven.util.SafeCloseable; +import org.jetbrains.annotations.NotNull; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.function.UnaryOperator; + +import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; + +/** + * An {@link IterativeChunkedAggregationOperator} used in the implementation of {@link Table#applyToAllBy}. + */ +class FormulaMultiColumnChunkedOperator implements IterativeChunkedAggregationOperator { + + private final QueryTable inputTable; + + private final GroupByChunkedOperator groupBy; + private final boolean delegateToBy; + private final SelectColumn selectColumn; + private final WritableColumnSource resultColumn; + private final String[] inputKeyColumns; + + private ChunkSource formulaDataSource; + + /** + * Converts the upstream MCS to a downstream MCS. + */ + private UnaryOperator inputToResultModifiedColumnSetFactory; + + /** + * Captures the upstream MCS in resetForStep to use in propagateUpdates. + */ + private ModifiedColumnSet updateUpstreamModifiedColumnSet; + + /** + * Construct an operator for applying a formula to slot-vectors over an aggregated table.. + * + * @param groupBy The {@link GroupByChunkedOperator} to use for tracking indices + * @param delegateToBy Whether this operator is responsible for passing methods through to {@code groupBy}. Should + * be false if {@code groupBy} is updated by the helper, or if this is not the first operator sharing + * {@code groupBy}. + * @param selectColumn The formula column that will produce the results + */ + FormulaMultiColumnChunkedOperator( + @NotNull final QueryTable inputTable, + @NotNull final GroupByChunkedOperator groupBy, + final boolean delegateToBy, + @NotNull final SelectColumn selectColumn, + @NotNull final String[] inputKeyColumns) { + this.inputTable = inputTable; + this.groupBy = groupBy; + this.delegateToBy = delegateToBy; + this.selectColumn = selectColumn; + this.inputKeyColumns = inputKeyColumns; + + resultColumn = ArrayBackedColumnSource.getMemoryColumnSource( + 0, selectColumn.getReturnedType(), selectColumn.getReturnedComponentType()); + } + + @Override + public void addChunk(final BucketedContext bucketedContext, final Chunk values, + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + if (delegateToBy) { + groupBy.addChunk(bucketedContext, values, inputRowKeys, destinations, startPositions, length, + stateModified); + } + } + + @Override + public void removeChunk(final BucketedContext bucketedContext, final Chunk values, + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + if (delegateToBy) { + groupBy.removeChunk(bucketedContext, values, inputRowKeys, destinations, startPositions, length, + stateModified); + } + } + + @Override + public void modifyChunk(final BucketedContext bucketedContext, final Chunk previousValues, + final Chunk newValues, + @NotNull final LongChunk postShiftRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + if (delegateToBy) { + groupBy.modifyChunk(bucketedContext, previousValues, newValues, postShiftRowKeys, destinations, + startPositions, + length, stateModified); + } + } + + @Override + public void shiftChunk(final BucketedContext bucketedContext, final Chunk previousValues, + final Chunk newValues, + @NotNull final LongChunk preShiftRowKeys, + @NotNull final LongChunk postShiftRowKeys, + @NotNull final IntChunk destinations, + @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + if (delegateToBy) { + groupBy.shiftChunk(bucketedContext, previousValues, newValues, preShiftRowKeys, postShiftRowKeys, + destinations, + startPositions, length, stateModified); + } + } + + @Override + public void modifyRowKeys(final BucketedContext context, + @NotNull final LongChunk inputRowKeys, + @NotNull final IntChunk destinations, @NotNull final IntChunk startPositions, + @NotNull final IntChunk length, @NotNull final WritableBooleanChunk stateModified) { + if (delegateToBy) { + groupBy.modifyRowKeys(context, inputRowKeys, destinations, startPositions, length, stateModified); + } + } + + @Override + public boolean addChunk(final SingletonContext singletonContext, final int chunkSize, + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { + if (delegateToBy) { + return groupBy.addChunk(singletonContext, chunkSize, values, inputRowKeys, destination); + } else { + return false; + } + } + + @Override + public boolean removeChunk(final SingletonContext singletonContext, final int chunkSize, + final Chunk values, + @NotNull final LongChunk inputRowKeys, final long destination) { + if (delegateToBy) { + return groupBy.removeChunk(singletonContext, chunkSize, values, inputRowKeys, destination); + } else { + return false; + } + } + + @Override + public boolean modifyChunk(final SingletonContext singletonContext, final int chunkSize, + final Chunk previousValues, final Chunk newValues, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { + if (delegateToBy) { + return groupBy.modifyChunk(singletonContext, chunkSize, previousValues, newValues, postShiftRowKeys, + destination); + } else { + return false; + } + } + + @Override + public boolean shiftChunk(final SingletonContext singletonContext, final Chunk previousValues, + final Chunk newValues, + @NotNull final LongChunk preShiftRowKeys, + @NotNull final LongChunk postShiftRowKeys, + final long destination) { + if (delegateToBy) { + return groupBy.shiftChunk(singletonContext, previousValues, newValues, preShiftRowKeys, postShiftRowKeys, + destination); + } else { + return false; + } + } + + @Override + public boolean modifyRowKeys(final SingletonContext context, + @NotNull final LongChunk rowKeys, + final long destination) { + if (delegateToBy) { + return groupBy.modifyRowKeys(context, rowKeys, destination); + } else { + return false; + } + } + + @Override + public boolean requiresRowKeys() { + return delegateToBy; + } + + @Override + public void ensureCapacity(final long tableSize) { + if (delegateToBy) { + groupBy.ensureCapacity(tableSize); + } + resultColumn.ensureCapacity(tableSize); + } + + @Override + public Map> getResultColumns() { + return Map.of(selectColumn.getName(), resultColumn); + } + + @Override + public void propagateInitialState(@NotNull final QueryTable resultTable, int startingDestinationsCount) { + if (delegateToBy) { + groupBy.propagateInitialState(resultTable, startingDestinationsCount); + } + + final Map> sourceColumns; + if (inputKeyColumns.length == 0) { + // noinspection unchecked + sourceColumns = (Map>) groupBy.getInputResultColumns(); + } else { + final Map> columnSourceMap = resultTable.getColumnSourceMap(); + sourceColumns = new HashMap<>(groupBy.getInputResultColumns()); + Arrays.stream(inputKeyColumns).forEach(col -> sourceColumns.put(col, columnSourceMap.get(col))); + } + selectColumn.initInputs(resultTable.getRowSet(), sourceColumns); + formulaDataSource = selectColumn.getDataView(); + + try (final DataCopyContext dataCopyContext = new DataCopyContext()) { + dataCopyContext.copyData(resultTable.getRowSet()); + } + } + + @Override + public void startTrackingPrevValues() { + if (delegateToBy) { + groupBy.startTrackingPrevValues(); + } + resultColumn.startTrackingPrevValues(); + } + + @Override + public UnaryOperator initializeRefreshing(@NotNull final QueryTable resultTable, + @NotNull final LivenessReferent aggregationUpdateListener) { + if (delegateToBy) { + // We cannot use the groupBy's result MCS factory, because the result column names are not + // guaranteed to be the same. + groupBy.initializeRefreshing(resultTable, aggregationUpdateListener); + } + + // Note that we also use the factory in propagateUpdates to identify the set of modified columns to handle. + if (selectColumn.getColumns().isEmpty()) { + return inputToResultModifiedColumnSetFactory = input -> ModifiedColumnSet.EMPTY; + } + final ModifiedColumnSet resultMCS = resultTable.newModifiedColumnSet(selectColumn.getName()); + final String[] inputColumnNames = selectColumn.getColumns().toArray(String[]::new); + final ModifiedColumnSet inputMCS = inputTable.newModifiedColumnSet(inputColumnNames); + return inputToResultModifiedColumnSetFactory = input -> { + if (groupBy.getSomeKeyHasAddsOrRemoves() || + (groupBy.getSomeKeyHasModifies() && input.containsAny(inputMCS))) { + return resultMCS; + } + return ModifiedColumnSet.EMPTY; + }; + } + + @Override + public void resetForStep(@NotNull final TableUpdate upstream, final int startingDestinationsCount) { + if (delegateToBy) { + groupBy.resetForStep(upstream, startingDestinationsCount); + } + updateUpstreamModifiedColumnSet = + upstream.modified().isEmpty() ? ModifiedColumnSet.EMPTY : upstream.modifiedColumnSet(); + } + + @Override + public void propagateUpdates(@NotNull final TableUpdate downstream, + @NotNull final RowSet newDestinations) { + if (delegateToBy) { + groupBy.propagateUpdates(downstream, newDestinations); + } + final ModifiedColumnSet resultModifiedColumnSet = + inputToResultModifiedColumnSetFactory.apply(updateUpstreamModifiedColumnSet); + updateUpstreamModifiedColumnSet = null; + + final boolean addsToProcess = downstream.added().isNonempty(); + final boolean modifiesToProcess = downstream.modified().isNonempty() && resultModifiedColumnSet.nonempty(); + final boolean removesToProcess = downstream.removed().isNonempty(); + + if (!addsToProcess && !modifiesToProcess && !removesToProcess) { + // Nothing to do. + return; + } + + if (!addsToProcess && !modifiesToProcess) { + // Only removes to handle, clear the removed objects. + if (!resultColumn.getType().isPrimitive()) { + try (final DataFillerContext dataFillerContext = new DataFillerContext()) { + dataFillerContext.clearObjectColumnData(downstream.removed()); + } + } + return; + } + + try (final DataCopyContext dataCopyContext = new DataCopyContext()) { + // Clear removed objects from the result column. + if (removesToProcess && !resultColumn.getType().isPrimitive()) { + dataCopyContext.clearObjectColumnData(downstream.removed()); + } + if (modifiesToProcess) { + dataCopyContext.copyData(downstream.modified()); + } + if (addsToProcess) { + dataCopyContext.copyData(downstream.added()); + } + } + } + + @Override + public void propagateFailure(@NotNull final Throwable originalException, + @NotNull final TableListener.Entry sourceEntry) { + if (delegateToBy) { + groupBy.propagateFailure(originalException, sourceEntry); + } + } + + @Override + public BucketedContext makeBucketedContext(final int size) { + return delegateToBy ? groupBy.makeBucketedContext(size) : null; + } + + @Override + public SingletonContext makeSingletonContext(final int size) { + return delegateToBy ? groupBy.makeSingletonContext(size) : null; + } + + /** + * Helper class to efficiently clear object data from the result columns. + */ + private class DataFillerContext implements SafeCloseable { + + final FillFromContext fillFromContext; + + private DataFillerContext() { + fillFromContext = resultColumn.makeFillFromContext(BLOCK_SIZE); + } + + void clearObjectColumnData(@NotNull final RowSequence rowSequence) { + try (final RowSequence.Iterator rowSequenceIterator = rowSequence.getRowSequenceIterator(); + final WritableObjectChunk nullValueChunk = + WritableObjectChunk.makeWritableChunk(BLOCK_SIZE)) { + nullValueChunk.fillWithNullValue(0, BLOCK_SIZE); + while (rowSequenceIterator.hasMore()) { + final RowSequence rowSequenceSlice = rowSequenceIterator.getNextRowSequenceThrough( + calculateContainingBlockLastKey(rowSequenceIterator.peekNextKey())); + nullValueChunk.setSize(rowSequenceSlice.intSize()); + resultColumn.fillFromChunk(fillFromContext, nullValueChunk, rowSequenceSlice); + } + } + } + + @Override + public void close() { + fillFromContext.close(); + } + } + + /** + * Helper class to efficiently copy data from the formula data source to the result column. + */ + private class DataCopyContext extends DataFillerContext { + + private final SharedContext sharedContext; + private final GetContext getContext; + + private DataCopyContext() { + sharedContext = SharedContext.makeSharedContext(); + getContext = formulaDataSource.makeGetContext(BLOCK_SIZE, sharedContext); + } + + private void copyData(@NotNull final RowSequence rowSequence) { + try (final RowSequence.Iterator rowSequenceIterator = rowSequence.getRowSequenceIterator()) { + while (rowSequenceIterator.hasMore()) { + final RowSequence rowSequenceSlice = rowSequenceIterator.getNextRowSequenceThrough( + calculateContainingBlockLastKey(rowSequenceIterator.peekNextKey())); + resultColumn.fillFromChunk(fillFromContext, + formulaDataSource.getChunk(getContext, rowSequenceSlice), rowSequenceSlice); + sharedContext.reset(); + } + } + } + + @Override + public void close() { + super.close(); + sharedContext.close(); + getContext.close(); + } + } + + private static long calculateContainingBlockLastKey(final long firstKey) { + return (firstKey / BLOCK_SIZE) * BLOCK_SIZE + BLOCK_SIZE - 1; + } +} diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java index 86d12e827de..398a10cb074 100644 --- a/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java +++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/by/GroupByChunkedOperator.java @@ -26,7 +26,6 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.function.UnaryOperator; -import java.util.stream.Collectors; import static io.deephaven.engine.table.impl.sources.ArrayBackedColumnSource.BLOCK_SIZE; @@ -46,6 +45,7 @@ public final class GroupByChunkedOperator implements IterativeChunkedAggregation private final ObjectArraySource removedBuilders; private final String[] inputColumnNames; + private final Map> inputAggregatedColumns; private final Map> resultAggregatedColumns; private final ModifiedColumnSet aggregationInputsModifiedColumnSet; @@ -68,10 +68,16 @@ public GroupByChunkedOperator( live = inputTable.isRefreshing(); rowSets = new ObjectArraySource<>(WritableRowSet.class); addedBuilders = new ObjectArraySource<>(Object.class); - resultAggregatedColumns = Arrays.stream(aggregatedColumnPairs).collect(Collectors.toMap(MatchPair::leftColumn, - matchPair -> AggregateColumnSource - .make(inputTable.getColumnSource(matchPair.rightColumn()), rowSets), - Assert::neverInvoked, LinkedHashMap::new)); + + inputAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); + resultAggregatedColumns = new LinkedHashMap<>(aggregatedColumnPairs.length); + Arrays.stream(aggregatedColumnPairs).forEach(pair -> { + final AggregateColumnSource aggregateColumnSource = + AggregateColumnSource.make(inputTable.getColumnSource(pair.rightColumn()), rowSets); + inputAggregatedColumns.put(pair.rightColumn(), aggregateColumnSource); + resultAggregatedColumns.put(pair.leftColumn(), aggregateColumnSource); + }); + if (exposeRowSetsAs != null && resultAggregatedColumns.containsKey(exposeRowSetsAs)) { throw new IllegalArgumentException(String.format( "Exposing group RowSets as %s, but this conflicts with a requested grouped output column name", @@ -386,6 +392,13 @@ public void ensureCapacity(final long tableSize) { return resultAggregatedColumns; } + /** + * Get a map from input column names to the corresponding output {@link ColumnSource}. + */ + public Map> getInputResultColumns() { + return inputAggregatedColumns; + } + @Override public void startTrackingPrevValues() { // NB: We don't need previous tracking on the rowSets ColumnSource, even if it's exposed. It's in destination @@ -614,4 +627,12 @@ public boolean requiresRowKeys() { public boolean unchunkedRowSet() { return true; } + + boolean getSomeKeyHasAddsOrRemoves() { + return someKeyHasAddsOrRemoves; + } + + boolean getSomeKeyHasModifies() { + return someKeyHasModifies; + } } 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 8ed956ac6e4..072db53b3c1 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 @@ -40,11 +40,11 @@ public static void validate( } } - private boolean lastSupported; + private boolean hasUnsupportedAggs; private boolean isSupported(@NotNull final Aggregation aggregation) { aggregation.walk(this); - return lastSupported; + return !hasUnsupportedAggs; } @Override @@ -54,31 +54,36 @@ public void visit(@NotNull final Aggregations aggregations) { @Override public void visit(@NotNull final ColumnAggregation columnAgg) { - lastSupported = columnAgg.spec() instanceof AggSpecGroup; + hasUnsupportedAggs |= !(columnAgg.spec() instanceof AggSpecGroup); } @Override public void visit(@NotNull final ColumnAggregations columnAggs) { - lastSupported = columnAggs.spec() instanceof AggSpecGroup; + hasUnsupportedAggs |= !(columnAggs.spec() instanceof AggSpecGroup); } @Override public void visit(@NotNull final Count count) { - lastSupported = false; + hasUnsupportedAggs = true; } @Override public void visit(@NotNull final FirstRowKey firstRowKey) { - lastSupported = false; + hasUnsupportedAggs = true; } @Override public void visit(@NotNull final LastRowKey lastRowKey) { - lastSupported = false; + hasUnsupportedAggs = true; } @Override public void visit(@NotNull final Partition partition) { - lastSupported = false; + hasUnsupportedAggs = true; + } + + @Override + public void visit(@NotNull final Formula formula) { + hasUnsupportedAggs = true; } } 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 41c03989f6c..e917d44885f 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 @@ -10,11 +10,14 @@ import io.deephaven.api.agg.Aggregation; import io.deephaven.api.agg.spec.AggSpec; import io.deephaven.api.object.UnionObject; +import io.deephaven.base.verify.Assert; import io.deephaven.chunk.IntChunk; import io.deephaven.chunk.attributes.Values; import io.deephaven.engine.context.ExecutionContext; import io.deephaven.engine.rowset.RowSet; +import io.deephaven.engine.rowset.RowSetShiftData; import io.deephaven.engine.table.ColumnSource; +import io.deephaven.engine.table.ModifiedColumnSet; import io.deephaven.engine.table.Table; import io.deephaven.engine.table.impl.util.ColumnHolder; import io.deephaven.engine.table.vectors.ColumnVectors; @@ -60,7 +63,8 @@ public void setUp() throws Exception { public void testBy() { 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); - Table table = TableTools.newTable(aHolder, bHolder); + ColumnHolder cHolder = col("C", 1, 1, 1, 1, 1, 1, 1, 1, 1, 1); + Table table = TableTools.newTable(aHolder, bHolder, cHolder); show(table); assertEquals(10, table.size()); assertEquals(2, table.groupBy("A").size()); @@ -68,16 +72,52 @@ public void testBy() { Table minMax = table.aggBy( List.of( AggFormula("min(each)", "each", "Min=B"), - AggFormula("max(each)", "each", "Max=B")), + AggFormula("max(each)", "each", "Max=B"), + AggFormula("sum(each)", "each", "Sum=B"), + AggFormula("f_const=6.0 + 3"), + AggFormula("f_min=min(B)"), + AggFormula("f_max=max(B)"), + AggFormula("f_sum=sum(B)"), + AggFormula("f_sum_two_col=sum(B) + sum(C)"), + AggFormula("f_custom_sum=A * (sum(B) + sum(C))"), + AggFormula("f_weighted_avg=wavg(B, C)")), "A"); show(minMax); + assertEquals(2, minMax.size()); + + DoubleVector consts = ColumnVectors.ofDouble(minMax, "f_const"); + assertEquals(9.0, consts.get(0)); + assertEquals(9.0, consts.get(1)); + IntVector mins = ColumnVectors.ofInt(minMax, "Min"); assertEquals(1, mins.get(0)); assertEquals(3, mins.get(1)); + mins = ColumnVectors.ofInt(minMax, "f_min"); + assertEquals(1, mins.get(0)); + assertEquals(3, mins.get(1)); + IntVector maxes = ColumnVectors.ofInt(minMax, "Max"); assertEquals(10, maxes.get(0)); assertEquals(8, maxes.get(1)); + maxes = ColumnVectors.ofInt(minMax, "f_max"); + assertEquals(10, maxes.get(0)); + assertEquals(8, maxes.get(1)); + + LongVector sums = ColumnVectors.ofLong(minMax, "Sum"); + assertEquals(33, sums.get(0)); + assertEquals(22, sums.get(1)); + sums = ColumnVectors.ofLong(minMax, "f_sum"); + assertEquals(33, sums.get(0)); + assertEquals(22, sums.get(1)); + + sums = ColumnVectors.ofLong(minMax, "f_sum_two_col"); + assertEquals(33 + 6, sums.get(0)); + assertEquals(22 + 4, sums.get(1)); + + sums = ColumnVectors.ofLong(minMax, "f_custom_sum"); + assertEquals(0, sums.get(0)); + assertEquals(22 + 4, sums.get(1)); Table doubleCounted = table.aggBy(List.of(AggCount("Count1"), AggCount("Count2")), "A"); show(doubleCounted); @@ -127,8 +167,8 @@ public void testBy() { for (int ii = 0; ii < bChunk.size(); ++ii) { doubles[ii] = 1.1 * bChunk.get(ii); } - ColumnHolder cHolder = col("C", doubles); - table = TableTools.newTable(aHolder, bHolder, cHolder); + ColumnHolder dHolder = col("D", doubles); + table = TableTools.newTable(aHolder, bHolder, cHolder, dHolder); show(table); Table summary = table.aggBy(summaryStatistics, "A"); show(summary); @@ -215,7 +255,16 @@ public Table e() { public Table e() { return queryTable.aggBy(List.of( AggFormula("min(each)", "each", "MinI=intCol", "MinD=doubleCol"), - AggFormula("max(each)", "each", "MaxI=intCol")), "Sym").sort("Sym"); + AggFormula("max(each)", "each", "MaxI=intCol"), + AggFormula("f_const=6.0 / 3"), + AggFormula("f_min=min(intColNulls)"), + AggFormula("f_max=max(doubleColNulls)"), + AggFormula("f_sum=sum(intColNulls + doubleColNulls)"), + AggFormula("f_key=Sym.equals(\"a\") ? \"a\" : \"not a\""), + AggFormula("f_key_sum=Sym + ':' + sum(intColNulls + doubleColNulls)"), + AggFormula( + "f_custom_sum=sum(intColNulls) + sum(doubleCol) + min(doubleColNulls)")), + "Sym").sort("Sym"); } }, new QueryTableTest.TableComparator( @@ -832,4 +881,179 @@ public void testAggAllByWithFormatColumn() { cs = result.getColumnSource("Integers"); assertEquals(1, cs.get(0)); } + + @Test + public void testFormulaUpdatePropagation() { + final QueryTable table = TstUtils.testRefreshingTable( + i(2, 4, 6).toTracking(), + col("Key", "A", "B", "A"), + doubleCol("Double", 1.0, 2.0, 2.0), + shortCol("Short", (short) 10, (short) 20, (short) 30), + intCol("Int", 2, 4, 6)); + final QueryTable result = (QueryTable) table.aggBy( + List.of(AggFormula("sumInt=sum(Int)"), + AggFormula("sumDouble=sum(Double)"), + AggFormula("sumCustom=sum(Int) + sum(Double)")), + "Key"); + + // Add to "B" bucket + final ControlledUpdateGraph updateGraph = ExecutionContext.getContext().getUpdateGraph().cast(); + + final SimpleListener resultListener = new SimpleListener(result); + result.addUpdateListener(resultListener); + int lastResultListenerCount = 0; + + final ModifiedColumnSet sumIntMCS = result.newModifiedColumnSet("sumInt"); + final ModifiedColumnSet sumDoubleMCS = result.newModifiedColumnSet("sumDouble"); + final ModifiedColumnSet sumCustomMCS = result.newModifiedColumnSet("sumCustom"); + final ModifiedColumnSet allFormulaMCS = result.newModifiedColumnSet("sumInt", "sumDouble", "sumCustom"); + + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(8), + col("Key", "B"), + doubleCol("Double", 3.0), + shortCol("Short", (short) 40), + intCol("Int", 8)); // Add to "B" bucket + table.notifyListeners(i(8), i(), i()); + }); + Assert.eq(lastResultListenerCount + 1, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + Assert.eqTrue(resultListener.update.added().isEmpty(), "added() matches expected"); + Assert.eqTrue(resultListener.update.removed().isEmpty(), "removed() matches expected"); + Assert.eqTrue(resultListener.update.shifted().empty(), "shifted() matches expected"); + Assert.eqTrue(resultListener.update.modified().equals(i(1)), "modified() matches expected"); + + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAll(allFormulaMCS), "MCS matches expected"); + lastResultListenerCount = resultListener.count; + + // Change a row in the "B" bucket + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(8), + col("Key", "B"), + doubleCol("Double", 3.0), + shortCol("Short", (short) 40), + intCol("Int", 9)); // Change Int + + table.notifyListeners(new TableUpdateImpl( + i(), i(), i(8), + RowSetShiftData.EMPTY, + table.newModifiedColumnSet("Int"))); + }); + Assert.eq(lastResultListenerCount + 1, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + Assert.eqTrue(resultListener.update.added().isEmpty(), "added() matches expected"); + Assert.eqTrue(resultListener.update.removed().isEmpty(), "removed() matches expected"); + Assert.eqTrue(resultListener.update.shifted().empty(), "shifted() matches expected"); + Assert.eqTrue(resultListener.update.modified().equals(i(1)), "modified() matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAny(sumIntMCS), "MCS matches expected"); + Assert.eqFalse(resultListener.update.modifiedColumnSet().containsAny(sumDoubleMCS), "MCS matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAny(sumCustomMCS), "MCS matches expected"); + lastResultListenerCount = resultListener.count; + + // Change a row in the "B" bucket + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(8), + col("Key", "B"), + doubleCol("Double", 5.0), // Change Double + shortCol("Short", (short) 40), + intCol("Int", 9)); + + table.notifyListeners(new TableUpdateImpl( + i(), i(), i(8), + RowSetShiftData.EMPTY, + table.newModifiedColumnSet("Double"))); + }); + Assert.eq(lastResultListenerCount + 1, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + Assert.eqTrue(resultListener.update.added().isEmpty(), "added() matches expected"); + Assert.eqTrue(resultListener.update.removed().isEmpty(), "removed() matches expected"); + Assert.eqTrue(resultListener.update.shifted().empty(), "shifted() matches expected"); + Assert.eqTrue(resultListener.update.modified().equals(i(1)), "modified() matches expected"); + Assert.eqFalse(resultListener.update.modifiedColumnSet().containsAny(sumIntMCS), "MCS matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAny(sumDoubleMCS), "MCS matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAny(sumCustomMCS), "MCS matches expected"); + lastResultListenerCount = resultListener.count; + + // Change a row in the "B" bucket + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(8), + col("Key", "B"), + doubleCol("Double", 5.0), + shortCol("Short", (short) 50), // Change Short + intCol("Int", 9)); + + table.notifyListeners(new TableUpdateImpl( + i(), i(), i(8), + RowSetShiftData.EMPTY, + table.newModifiedColumnSet("Short"))); + }); + // No update expected from the result table, + Assert.eq(lastResultListenerCount, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + + // New "C" bucket + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(9), + col("Key", "C"), + doubleCol("Double", 4.0), + shortCol("Short", (short) 40), + intCol("Int", 10)); // New "C" bucket in isolation + table.notifyListeners(i(9), i(), i()); + }); + Assert.eq(lastResultListenerCount + 1, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + Assert.eqTrue(resultListener.update.added().equals(i(2)), "added() matches expected"); + Assert.eqTrue(resultListener.update.removed().isEmpty(), "removed() matches expected"); + Assert.eqTrue(resultListener.update.shifted().empty(), "shifted() matches expected"); + Assert.eqTrue(resultListener.update.modified().isEmpty(), "modified() matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().empty(), "MCS matches expected"); + lastResultListenerCount = resultListener.count; + + // Row from "B" bucket to "C" bucket + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(8), + col("Key", "C"), + doubleCol("Double", 3.0), + shortCol("Short", (short) 40), + intCol("Int", 11)); // Row from "B" bucket to "C" bucket + table.notifyListeners(new TableUpdateImpl( + i(), i(), i(8), + RowSetShiftData.EMPTY, + table.newModifiedColumnSet("Key"))); + }); + Assert.eq(lastResultListenerCount + 1, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + Assert.eqTrue(resultListener.update.added().isEmpty(), "added() matches expected"); + Assert.eqTrue(resultListener.update.removed().isEmpty(), "removed() matches expected"); + Assert.eqTrue(resultListener.update.shifted().empty(), "shifted() matches expected"); + Assert.eqTrue(resultListener.update.modified().equals(i(1, 2)), "modified() matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAll(allFormulaMCS), "MCS matches expected"); + lastResultListenerCount = resultListener.count; + + // New "D" bucket, and new row in "C" + updateGraph.runWithinUnitTestCycle(() -> { + TstUtils.addToTable(table, + i(10, 11), + col("Key", "D", "C"), + doubleCol("Double", 5.0, 6.0), + shortCol("Short", (short) 40, (short) 50), + intCol("Int", 10, 11)); // New "D" bucket + table.notifyListeners(i(10, 11), i(), i()); + }); + Assert.eq(lastResultListenerCount + 1, "lastResultListenerCount + 1", + resultListener.count, "resultListener.count"); + Assert.eqTrue(resultListener.update.added().equals(i(3)), "added() matches expected"); + Assert.eqTrue(resultListener.update.removed().isEmpty(), "removed() matches expected"); + Assert.eqTrue(resultListener.update.shifted().empty(), "shifted() matches expected"); + Assert.eqTrue(resultListener.update.modified().equals(i(2)), "modified() matches expected"); + Assert.eqTrue(resultListener.update.modifiedColumnSet().containsAll(allFormulaMCS), "MCS matches expected"); + + TableTools.show(result); + } } diff --git a/go/internal/proto/table/table.pb.go b/go/internal/proto/table/table.pb.go index 79cd00c29ab..106decee109 100644 --- a/go/internal/proto/table/table.pb.go +++ b/go/internal/proto/table/table.pb.go @@ -341,7 +341,7 @@ func (x MathContext_RoundingMode) Number() protoreflect.EnumNumber { // Deprecated: Use MathContext_RoundingMode.Descriptor instead. func (MathContext_RoundingMode) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{9, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{10, 0} } // Deprecated: Do not use. @@ -394,7 +394,7 @@ func (x AsOfJoinTablesRequest_MatchRule) Number() protoreflect.EnumNumber { // Deprecated: Use AsOfJoinTablesRequest_MatchRule.Descriptor instead. func (AsOfJoinTablesRequest_MatchRule) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{27, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{28, 0} } type RangeJoinTablesRequest_RangeStartRule int32 @@ -446,7 +446,7 @@ func (x RangeJoinTablesRequest_RangeStartRule) Number() protoreflect.EnumNumber // Deprecated: Use RangeJoinTablesRequest_RangeStartRule.Descriptor instead. func (RangeJoinTablesRequest_RangeStartRule) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{31, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{32, 0} } type RangeJoinTablesRequest_RangeEndRule int32 @@ -498,7 +498,7 @@ func (x RangeJoinTablesRequest_RangeEndRule) Number() protoreflect.EnumNumber { // Deprecated: Use RangeJoinTablesRequest_RangeEndRule.Descriptor instead. func (RangeJoinTablesRequest_RangeEndRule) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{31, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{32, 1} } type ComboAggregateRequest_AggType int32 @@ -580,7 +580,7 @@ func (x ComboAggregateRequest_AggType) Number() protoreflect.EnumNumber { // Deprecated: Use ComboAggregateRequest_AggType.Descriptor instead. func (ComboAggregateRequest_AggType) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{32, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{33, 0} } type SortDescriptor_SortDirection int32 @@ -632,7 +632,7 @@ func (x SortDescriptor_SortDirection) Number() protoreflect.EnumNumber { // Deprecated: Use SortDescriptor_SortDirection.Descriptor instead. func (SortDescriptor_SortDirection) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{38, 0} } type CompareCondition_CompareOperation int32 @@ -690,7 +690,7 @@ func (x CompareCondition_CompareOperation) Number() protoreflect.EnumNumber { // Deprecated: Use CompareCondition_CompareOperation.Descriptor instead. func (CompareCondition_CompareOperation) EnumDescriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{49, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{50, 0} } type TableReference struct { @@ -1335,6 +1335,77 @@ func (x *SelectOrUpdateRequest) GetColumnSpecs() []string { return nil } +type Selectable struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // message ColumnExpression { + // string column_name = 1; + // Expression expression = 2; + // } + // + // Types that are assignable to Type: + // *Selectable_Raw + Type isSelectable_Type `protobuf_oneof:"type"` +} + +func (x *Selectable) Reset() { + *x = Selectable{} + if protoimpl.UnsafeEnabled { + mi := &file_deephaven_proto_table_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Selectable) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Selectable) ProtoMessage() {} + +func (x *Selectable) ProtoReflect() protoreflect.Message { + mi := &file_deephaven_proto_table_proto_msgTypes[9] + 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 Selectable.ProtoReflect.Descriptor instead. +func (*Selectable) Descriptor() ([]byte, []int) { + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{9} +} + +func (m *Selectable) GetType() isSelectable_Type { + if m != nil { + return m.Type + } + return nil +} + +func (x *Selectable) GetRaw() string { + if x, ok := x.GetType().(*Selectable_Raw); ok { + return x.Raw + } + return "" +} + +type isSelectable_Type interface { + isSelectable_Type() +} + +type Selectable_Raw struct { + Raw string `protobuf:"bytes,1,opt,name=raw,proto3,oneof"` // ColumnExpression column_expression = 2; +} + +func (*Selectable_Raw) isSelectable_Type() {} + type MathContext struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1347,7 +1418,7 @@ type MathContext struct { func (x *MathContext) Reset() { *x = MathContext{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[9] + mi := &file_deephaven_proto_table_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1360,7 +1431,7 @@ func (x *MathContext) String() string { func (*MathContext) ProtoMessage() {} func (x *MathContext) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[9] + mi := &file_deephaven_proto_table_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1373,7 +1444,7 @@ func (x *MathContext) ProtoReflect() protoreflect.Message { // Deprecated: Use MathContext.ProtoReflect.Descriptor instead. func (*MathContext) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{9} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{10} } func (x *MathContext) GetPrecision() int32 { @@ -1405,7 +1476,7 @@ type UpdateByWindowScale struct { func (x *UpdateByWindowScale) Reset() { *x = UpdateByWindowScale{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[10] + mi := &file_deephaven_proto_table_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1418,7 +1489,7 @@ func (x *UpdateByWindowScale) String() string { func (*UpdateByWindowScale) ProtoMessage() {} func (x *UpdateByWindowScale) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[10] + mi := &file_deephaven_proto_table_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1431,7 +1502,7 @@ func (x *UpdateByWindowScale) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateByWindowScale.ProtoReflect.Descriptor instead. func (*UpdateByWindowScale) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{10} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{11} } func (m *UpdateByWindowScale) GetType() isUpdateByWindowScale_Type { @@ -1488,7 +1559,7 @@ type UpdateByEmOptions struct { func (x *UpdateByEmOptions) Reset() { *x = UpdateByEmOptions{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[11] + mi := &file_deephaven_proto_table_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1501,7 +1572,7 @@ func (x *UpdateByEmOptions) String() string { func (*UpdateByEmOptions) ProtoMessage() {} func (x *UpdateByEmOptions) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[11] + mi := &file_deephaven_proto_table_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1514,7 +1585,7 @@ func (x *UpdateByEmOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateByEmOptions.ProtoReflect.Descriptor instead. func (*UpdateByEmOptions) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{11} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{12} } func (x *UpdateByEmOptions) GetOnNullValue() BadDataBehavior { @@ -1571,7 +1642,7 @@ type UpdateByDeltaOptions struct { func (x *UpdateByDeltaOptions) Reset() { *x = UpdateByDeltaOptions{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[12] + mi := &file_deephaven_proto_table_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1584,7 +1655,7 @@ func (x *UpdateByDeltaOptions) String() string { func (*UpdateByDeltaOptions) ProtoMessage() {} func (x *UpdateByDeltaOptions) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[12] + mi := &file_deephaven_proto_table_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1597,7 +1668,7 @@ func (x *UpdateByDeltaOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateByDeltaOptions.ProtoReflect.Descriptor instead. func (*UpdateByDeltaOptions) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{12} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13} } func (x *UpdateByDeltaOptions) GetNullBehavior() UpdateByNullBehavior { @@ -1626,7 +1697,7 @@ type UpdateByRequest struct { func (x *UpdateByRequest) Reset() { *x = UpdateByRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[13] + mi := &file_deephaven_proto_table_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1639,7 +1710,7 @@ func (x *UpdateByRequest) String() string { func (*UpdateByRequest) ProtoMessage() {} func (x *UpdateByRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[13] + mi := &file_deephaven_proto_table_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1652,7 +1723,7 @@ func (x *UpdateByRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateByRequest.ProtoReflect.Descriptor instead. func (*UpdateByRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14} } func (x *UpdateByRequest) GetResultId() *ticket.Ticket { @@ -1703,7 +1774,7 @@ type SelectDistinctRequest struct { func (x *SelectDistinctRequest) Reset() { *x = SelectDistinctRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[14] + mi := &file_deephaven_proto_table_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1716,7 +1787,7 @@ func (x *SelectDistinctRequest) String() string { func (*SelectDistinctRequest) ProtoMessage() {} func (x *SelectDistinctRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[14] + mi := &file_deephaven_proto_table_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1729,7 +1800,7 @@ func (x *SelectDistinctRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SelectDistinctRequest.ProtoReflect.Descriptor instead. func (*SelectDistinctRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{15} } func (x *SelectDistinctRequest) GetResultId() *ticket.Ticket { @@ -1766,7 +1837,7 @@ type DropColumnsRequest struct { func (x *DropColumnsRequest) Reset() { *x = DropColumnsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[15] + mi := &file_deephaven_proto_table_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1779,7 +1850,7 @@ func (x *DropColumnsRequest) String() string { func (*DropColumnsRequest) ProtoMessage() {} func (x *DropColumnsRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[15] + mi := &file_deephaven_proto_table_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1792,7 +1863,7 @@ func (x *DropColumnsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DropColumnsRequest.ProtoReflect.Descriptor instead. func (*DropColumnsRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{15} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{16} } func (x *DropColumnsRequest) GetResultId() *ticket.Ticket { @@ -1829,7 +1900,7 @@ type UnstructuredFilterTableRequest struct { func (x *UnstructuredFilterTableRequest) Reset() { *x = UnstructuredFilterTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[16] + mi := &file_deephaven_proto_table_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1842,7 +1913,7 @@ func (x *UnstructuredFilterTableRequest) String() string { func (*UnstructuredFilterTableRequest) ProtoMessage() {} func (x *UnstructuredFilterTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[16] + mi := &file_deephaven_proto_table_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1855,7 +1926,7 @@ func (x *UnstructuredFilterTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UnstructuredFilterTableRequest.ProtoReflect.Descriptor instead. func (*UnstructuredFilterTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{16} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{17} } func (x *UnstructuredFilterTableRequest) GetResultId() *ticket.Ticket { @@ -1892,7 +1963,7 @@ type HeadOrTailRequest struct { func (x *HeadOrTailRequest) Reset() { *x = HeadOrTailRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[17] + mi := &file_deephaven_proto_table_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1905,7 +1976,7 @@ func (x *HeadOrTailRequest) String() string { func (*HeadOrTailRequest) ProtoMessage() {} func (x *HeadOrTailRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[17] + mi := &file_deephaven_proto_table_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1918,7 +1989,7 @@ func (x *HeadOrTailRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use HeadOrTailRequest.ProtoReflect.Descriptor instead. func (*HeadOrTailRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{17} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{18} } func (x *HeadOrTailRequest) GetResultId() *ticket.Ticket { @@ -1956,7 +2027,7 @@ type HeadOrTailByRequest struct { func (x *HeadOrTailByRequest) Reset() { *x = HeadOrTailByRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[18] + mi := &file_deephaven_proto_table_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1969,7 +2040,7 @@ func (x *HeadOrTailByRequest) String() string { func (*HeadOrTailByRequest) ProtoMessage() {} func (x *HeadOrTailByRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[18] + mi := &file_deephaven_proto_table_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1982,7 +2053,7 @@ func (x *HeadOrTailByRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use HeadOrTailByRequest.ProtoReflect.Descriptor instead. func (*HeadOrTailByRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{18} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{19} } func (x *HeadOrTailByRequest) GetResultId() *ticket.Ticket { @@ -2027,7 +2098,7 @@ type UngroupRequest struct { func (x *UngroupRequest) Reset() { *x = UngroupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[19] + mi := &file_deephaven_proto_table_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2040,7 +2111,7 @@ func (x *UngroupRequest) String() string { func (*UngroupRequest) ProtoMessage() {} func (x *UngroupRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[19] + mi := &file_deephaven_proto_table_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2053,7 +2124,7 @@ func (x *UngroupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UngroupRequest.ProtoReflect.Descriptor instead. func (*UngroupRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{19} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{20} } func (x *UngroupRequest) GetResultId() *ticket.Ticket { @@ -2097,7 +2168,7 @@ type MergeTablesRequest struct { func (x *MergeTablesRequest) Reset() { *x = MergeTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[20] + mi := &file_deephaven_proto_table_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2110,7 +2181,7 @@ func (x *MergeTablesRequest) String() string { func (*MergeTablesRequest) ProtoMessage() {} func (x *MergeTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[20] + mi := &file_deephaven_proto_table_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2123,7 +2194,7 @@ func (x *MergeTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MergeTablesRequest.ProtoReflect.Descriptor instead. func (*MergeTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{20} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{21} } func (x *MergeTablesRequest) GetResultId() *ticket.Ticket { @@ -2159,7 +2230,7 @@ type SnapshotTableRequest struct { func (x *SnapshotTableRequest) Reset() { *x = SnapshotTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[21] + mi := &file_deephaven_proto_table_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2172,7 +2243,7 @@ func (x *SnapshotTableRequest) String() string { func (*SnapshotTableRequest) ProtoMessage() {} func (x *SnapshotTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[21] + mi := &file_deephaven_proto_table_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2185,7 +2256,7 @@ func (x *SnapshotTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SnapshotTableRequest.ProtoReflect.Descriptor instead. func (*SnapshotTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{21} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{22} } func (x *SnapshotTableRequest) GetResultId() *ticket.Ticket { @@ -2225,7 +2296,7 @@ type SnapshotWhenTableRequest struct { func (x *SnapshotWhenTableRequest) Reset() { *x = SnapshotWhenTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[22] + mi := &file_deephaven_proto_table_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2238,7 +2309,7 @@ func (x *SnapshotWhenTableRequest) String() string { func (*SnapshotWhenTableRequest) ProtoMessage() {} func (x *SnapshotWhenTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[22] + mi := &file_deephaven_proto_table_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2251,7 +2322,7 @@ func (x *SnapshotWhenTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SnapshotWhenTableRequest.ProtoReflect.Descriptor instead. func (*SnapshotWhenTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{22} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{23} } func (x *SnapshotWhenTableRequest) GetResultId() *ticket.Ticket { @@ -2320,7 +2391,7 @@ type CrossJoinTablesRequest struct { func (x *CrossJoinTablesRequest) Reset() { *x = CrossJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[23] + mi := &file_deephaven_proto_table_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2333,7 +2404,7 @@ func (x *CrossJoinTablesRequest) String() string { func (*CrossJoinTablesRequest) ProtoMessage() {} func (x *CrossJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[23] + mi := &file_deephaven_proto_table_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2346,7 +2417,7 @@ func (x *CrossJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CrossJoinTablesRequest.ProtoReflect.Descriptor instead. func (*CrossJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{23} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{24} } func (x *CrossJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -2406,7 +2477,7 @@ type NaturalJoinTablesRequest struct { func (x *NaturalJoinTablesRequest) Reset() { *x = NaturalJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[24] + mi := &file_deephaven_proto_table_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2419,7 +2490,7 @@ func (x *NaturalJoinTablesRequest) String() string { func (*NaturalJoinTablesRequest) ProtoMessage() {} func (x *NaturalJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[24] + mi := &file_deephaven_proto_table_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2432,7 +2503,7 @@ func (x *NaturalJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use NaturalJoinTablesRequest.ProtoReflect.Descriptor instead. func (*NaturalJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{24} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{25} } func (x *NaturalJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -2485,7 +2556,7 @@ type ExactJoinTablesRequest struct { func (x *ExactJoinTablesRequest) Reset() { *x = ExactJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[25] + mi := &file_deephaven_proto_table_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2498,7 +2569,7 @@ func (x *ExactJoinTablesRequest) String() string { func (*ExactJoinTablesRequest) ProtoMessage() {} func (x *ExactJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[25] + mi := &file_deephaven_proto_table_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2511,7 +2582,7 @@ func (x *ExactJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExactJoinTablesRequest.ProtoReflect.Descriptor instead. func (*ExactJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{25} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{26} } func (x *ExactJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -2564,7 +2635,7 @@ type LeftJoinTablesRequest struct { func (x *LeftJoinTablesRequest) Reset() { *x = LeftJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[26] + mi := &file_deephaven_proto_table_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2577,7 +2648,7 @@ func (x *LeftJoinTablesRequest) String() string { func (*LeftJoinTablesRequest) ProtoMessage() {} func (x *LeftJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[26] + mi := &file_deephaven_proto_table_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2590,7 +2661,7 @@ func (x *LeftJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LeftJoinTablesRequest.ProtoReflect.Descriptor instead. func (*LeftJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{26} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{27} } func (x *LeftJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -2647,7 +2718,7 @@ type AsOfJoinTablesRequest struct { func (x *AsOfJoinTablesRequest) Reset() { *x = AsOfJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[27] + mi := &file_deephaven_proto_table_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2660,7 +2731,7 @@ func (x *AsOfJoinTablesRequest) String() string { func (*AsOfJoinTablesRequest) ProtoMessage() {} func (x *AsOfJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[27] + mi := &file_deephaven_proto_table_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2673,7 +2744,7 @@ func (x *AsOfJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AsOfJoinTablesRequest.ProtoReflect.Descriptor instead. func (*AsOfJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{27} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{28} } func (x *AsOfJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -2739,7 +2810,7 @@ type AjRajTablesRequest struct { func (x *AjRajTablesRequest) Reset() { *x = AjRajTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[28] + mi := &file_deephaven_proto_table_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2752,7 +2823,7 @@ func (x *AjRajTablesRequest) String() string { func (*AjRajTablesRequest) ProtoMessage() {} func (x *AjRajTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[28] + mi := &file_deephaven_proto_table_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2765,7 +2836,7 @@ func (x *AjRajTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AjRajTablesRequest.ProtoReflect.Descriptor instead. func (*AjRajTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{28} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{29} } func (x *AjRajTablesRequest) GetResultId() *ticket.Ticket { @@ -2826,7 +2897,7 @@ type MultiJoinInput struct { func (x *MultiJoinInput) Reset() { *x = MultiJoinInput{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[29] + mi := &file_deephaven_proto_table_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2839,7 +2910,7 @@ func (x *MultiJoinInput) String() string { func (*MultiJoinInput) ProtoMessage() {} func (x *MultiJoinInput) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[29] + mi := &file_deephaven_proto_table_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2852,7 +2923,7 @@ func (x *MultiJoinInput) ProtoReflect() protoreflect.Message { // Deprecated: Use MultiJoinInput.ProtoReflect.Descriptor instead. func (*MultiJoinInput) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{29} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{30} } func (x *MultiJoinInput) GetSourceId() *TableReference { @@ -2889,7 +2960,7 @@ type MultiJoinTablesRequest struct { func (x *MultiJoinTablesRequest) Reset() { *x = MultiJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[30] + mi := &file_deephaven_proto_table_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2902,7 +2973,7 @@ func (x *MultiJoinTablesRequest) String() string { func (*MultiJoinTablesRequest) ProtoMessage() {} func (x *MultiJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[30] + mi := &file_deephaven_proto_table_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2915,7 +2986,7 @@ func (x *MultiJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MultiJoinTablesRequest.ProtoReflect.Descriptor instead. func (*MultiJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{30} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{31} } func (x *MultiJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -2957,7 +3028,7 @@ type RangeJoinTablesRequest struct { func (x *RangeJoinTablesRequest) Reset() { *x = RangeJoinTablesRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[31] + mi := &file_deephaven_proto_table_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2970,7 +3041,7 @@ func (x *RangeJoinTablesRequest) String() string { func (*RangeJoinTablesRequest) ProtoMessage() {} func (x *RangeJoinTablesRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[31] + mi := &file_deephaven_proto_table_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2983,7 +3054,7 @@ func (x *RangeJoinTablesRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RangeJoinTablesRequest.ProtoReflect.Descriptor instead. func (*RangeJoinTablesRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{31} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{32} } func (x *RangeJoinTablesRequest) GetResultId() *ticket.Ticket { @@ -3079,7 +3150,7 @@ type ComboAggregateRequest struct { func (x *ComboAggregateRequest) Reset() { *x = ComboAggregateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[32] + mi := &file_deephaven_proto_table_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3092,7 +3163,7 @@ func (x *ComboAggregateRequest) String() string { func (*ComboAggregateRequest) ProtoMessage() {} func (x *ComboAggregateRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[32] + mi := &file_deephaven_proto_table_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3105,7 +3176,7 @@ func (x *ComboAggregateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ComboAggregateRequest.ProtoReflect.Descriptor instead. func (*ComboAggregateRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{32} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{33} } func (x *ComboAggregateRequest) GetResultId() *ticket.Ticket { @@ -3157,7 +3228,7 @@ type AggregateAllRequest struct { func (x *AggregateAllRequest) Reset() { *x = AggregateAllRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[33] + mi := &file_deephaven_proto_table_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3170,7 +3241,7 @@ func (x *AggregateAllRequest) String() string { func (*AggregateAllRequest) ProtoMessage() {} func (x *AggregateAllRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[33] + mi := &file_deephaven_proto_table_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3183,7 +3254,7 @@ func (x *AggregateAllRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AggregateAllRequest.ProtoReflect.Descriptor instead. func (*AggregateAllRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{33} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34} } func (x *AggregateAllRequest) GetResultId() *ticket.Ticket { @@ -3249,7 +3320,7 @@ type AggSpec struct { func (x *AggSpec) Reset() { *x = AggSpec{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[34] + mi := &file_deephaven_proto_table_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3262,7 +3333,7 @@ func (x *AggSpec) String() string { func (*AggSpec) ProtoMessage() {} func (x *AggSpec) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[34] + mi := &file_deephaven_proto_table_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3275,7 +3346,7 @@ func (x *AggSpec) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec.ProtoReflect.Descriptor instead. func (*AggSpec) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35} } func (m *AggSpec) GetType() isAggSpec_Type { @@ -3613,7 +3684,7 @@ type AggregateRequest struct { func (x *AggregateRequest) Reset() { *x = AggregateRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[35] + mi := &file_deephaven_proto_table_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3626,7 +3697,7 @@ func (x *AggregateRequest) String() string { func (*AggregateRequest) ProtoMessage() {} func (x *AggregateRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[35] + mi := &file_deephaven_proto_table_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3639,7 +3710,7 @@ func (x *AggregateRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AggregateRequest.ProtoReflect.Descriptor instead. func (*AggregateRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{36} } func (x *AggregateRequest) GetResultId() *ticket.Ticket { @@ -3695,13 +3766,14 @@ type Aggregation struct { // *Aggregation_FirstRowKey // *Aggregation_LastRowKey // *Aggregation_Partition + // *Aggregation_Formula Type isAggregation_Type `protobuf_oneof:"type"` } func (x *Aggregation) Reset() { *x = Aggregation{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[36] + mi := &file_deephaven_proto_table_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3714,7 +3786,7 @@ func (x *Aggregation) String() string { func (*Aggregation) ProtoMessage() {} func (x *Aggregation) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[36] + mi := &file_deephaven_proto_table_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3727,7 +3799,7 @@ func (x *Aggregation) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation.ProtoReflect.Descriptor instead. func (*Aggregation) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{36} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37} } func (m *Aggregation) GetType() isAggregation_Type { @@ -3772,6 +3844,13 @@ func (x *Aggregation) GetPartition() *Aggregation_AggregationPartition { return nil } +func (x *Aggregation) GetFormula() *Aggregation_AggregationFormula { + if x, ok := x.GetType().(*Aggregation_Formula); ok { + return x.Formula + } + return nil +} + type isAggregation_Type interface { isAggregation_Type() } @@ -3796,6 +3875,10 @@ type Aggregation_Partition struct { Partition *Aggregation_AggregationPartition `protobuf:"bytes,5,opt,name=partition,proto3,oneof"` } +type Aggregation_Formula struct { + Formula *Aggregation_AggregationFormula `protobuf:"bytes,6,opt,name=formula,proto3,oneof"` +} + func (*Aggregation_Columns) isAggregation_Type() {} func (*Aggregation_Count) isAggregation_Type() {} @@ -3806,6 +3889,8 @@ func (*Aggregation_LastRowKey) isAggregation_Type() {} func (*Aggregation_Partition) isAggregation_Type() {} +func (*Aggregation_Formula) isAggregation_Type() {} + type SortDescriptor struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3819,7 +3904,7 @@ type SortDescriptor struct { func (x *SortDescriptor) Reset() { *x = SortDescriptor{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[37] + mi := &file_deephaven_proto_table_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3832,7 +3917,7 @@ func (x *SortDescriptor) String() string { func (*SortDescriptor) ProtoMessage() {} func (x *SortDescriptor) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[37] + mi := &file_deephaven_proto_table_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3845,7 +3930,7 @@ func (x *SortDescriptor) ProtoReflect() protoreflect.Message { // Deprecated: Use SortDescriptor.ProtoReflect.Descriptor instead. func (*SortDescriptor) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{38} } func (x *SortDescriptor) GetColumnName() string { @@ -3882,7 +3967,7 @@ type SortTableRequest struct { func (x *SortTableRequest) Reset() { *x = SortTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[38] + mi := &file_deephaven_proto_table_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3895,7 +3980,7 @@ func (x *SortTableRequest) String() string { func (*SortTableRequest) ProtoMessage() {} func (x *SortTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[38] + mi := &file_deephaven_proto_table_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3908,7 +3993,7 @@ func (x *SortTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SortTableRequest.ProtoReflect.Descriptor instead. func (*SortTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{38} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{39} } func (x *SortTableRequest) GetResultId() *ticket.Ticket { @@ -3945,7 +4030,7 @@ type FilterTableRequest struct { func (x *FilterTableRequest) Reset() { *x = FilterTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[39] + mi := &file_deephaven_proto_table_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3958,7 +4043,7 @@ func (x *FilterTableRequest) String() string { func (*FilterTableRequest) ProtoMessage() {} func (x *FilterTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[39] + mi := &file_deephaven_proto_table_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3971,7 +4056,7 @@ func (x *FilterTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use FilterTableRequest.ProtoReflect.Descriptor instead. func (*FilterTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{39} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{40} } func (x *FilterTableRequest) GetResultId() *ticket.Ticket { @@ -4012,7 +4097,7 @@ type SeekRowRequest struct { func (x *SeekRowRequest) Reset() { *x = SeekRowRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[40] + mi := &file_deephaven_proto_table_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4025,7 +4110,7 @@ func (x *SeekRowRequest) String() string { func (*SeekRowRequest) ProtoMessage() {} func (x *SeekRowRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[40] + mi := &file_deephaven_proto_table_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4038,7 +4123,7 @@ func (x *SeekRowRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SeekRowRequest.ProtoReflect.Descriptor instead. func (*SeekRowRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{40} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{41} } func (x *SeekRowRequest) GetSourceId() *ticket.Ticket { @@ -4101,7 +4186,7 @@ type SeekRowResponse struct { func (x *SeekRowResponse) Reset() { *x = SeekRowResponse{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[41] + mi := &file_deephaven_proto_table_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4114,7 +4199,7 @@ func (x *SeekRowResponse) String() string { func (*SeekRowResponse) ProtoMessage() {} func (x *SeekRowResponse) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[41] + mi := &file_deephaven_proto_table_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4127,7 +4212,7 @@ func (x *SeekRowResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use SeekRowResponse.ProtoReflect.Descriptor instead. func (*SeekRowResponse) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{41} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{42} } func (x *SeekRowResponse) GetResultRow() int64 { @@ -4148,7 +4233,7 @@ type Reference struct { func (x *Reference) Reset() { *x = Reference{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[42] + mi := &file_deephaven_proto_table_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4161,7 +4246,7 @@ func (x *Reference) String() string { func (*Reference) ProtoMessage() {} func (x *Reference) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[42] + mi := &file_deephaven_proto_table_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4174,7 +4259,7 @@ func (x *Reference) ProtoReflect() protoreflect.Message { // Deprecated: Use Reference.ProtoReflect.Descriptor instead. func (*Reference) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{42} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{43} } func (x *Reference) GetColumnName() string { @@ -4201,7 +4286,7 @@ type Literal struct { func (x *Literal) Reset() { *x = Literal{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[43] + mi := &file_deephaven_proto_table_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4214,7 +4299,7 @@ func (x *Literal) String() string { func (*Literal) ProtoMessage() {} func (x *Literal) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[43] + mi := &file_deephaven_proto_table_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4227,7 +4312,7 @@ func (x *Literal) ProtoReflect() protoreflect.Message { // Deprecated: Use Literal.ProtoReflect.Descriptor instead. func (*Literal) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{43} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{44} } func (m *Literal) GetValue() isLiteral_Value { @@ -4321,7 +4406,7 @@ type Value struct { func (x *Value) Reset() { *x = Value{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[44] + mi := &file_deephaven_proto_table_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4334,7 +4419,7 @@ func (x *Value) String() string { func (*Value) ProtoMessage() {} func (x *Value) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[44] + mi := &file_deephaven_proto_table_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4347,7 +4432,7 @@ func (x *Value) ProtoReflect() protoreflect.Message { // Deprecated: Use Value.ProtoReflect.Descriptor instead. func (*Value) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{44} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{45} } func (m *Value) GetData() isValue_Data { @@ -4409,7 +4494,7 @@ type Condition struct { func (x *Condition) Reset() { *x = Condition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[45] + mi := &file_deephaven_proto_table_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4422,7 +4507,7 @@ func (x *Condition) String() string { func (*Condition) ProtoMessage() {} func (x *Condition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[45] + mi := &file_deephaven_proto_table_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4435,7 +4520,7 @@ func (x *Condition) ProtoReflect() protoreflect.Message { // Deprecated: Use Condition.ProtoReflect.Descriptor instead. func (*Condition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{45} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{46} } func (m *Condition) GetData() isCondition_Data { @@ -4591,7 +4676,7 @@ type AndCondition struct { func (x *AndCondition) Reset() { *x = AndCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[46] + mi := &file_deephaven_proto_table_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4604,7 +4689,7 @@ func (x *AndCondition) String() string { func (*AndCondition) ProtoMessage() {} func (x *AndCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[46] + mi := &file_deephaven_proto_table_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4617,7 +4702,7 @@ func (x *AndCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use AndCondition.ProtoReflect.Descriptor instead. func (*AndCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{46} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{47} } func (x *AndCondition) GetFilters() []*Condition { @@ -4638,7 +4723,7 @@ type OrCondition struct { func (x *OrCondition) Reset() { *x = OrCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[47] + mi := &file_deephaven_proto_table_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4651,7 +4736,7 @@ func (x *OrCondition) String() string { func (*OrCondition) ProtoMessage() {} func (x *OrCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[47] + mi := &file_deephaven_proto_table_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4664,7 +4749,7 @@ func (x *OrCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use OrCondition.ProtoReflect.Descriptor instead. func (*OrCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{47} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{48} } func (x *OrCondition) GetFilters() []*Condition { @@ -4685,7 +4770,7 @@ type NotCondition struct { func (x *NotCondition) Reset() { *x = NotCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[48] + mi := &file_deephaven_proto_table_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4698,7 +4783,7 @@ func (x *NotCondition) String() string { func (*NotCondition) ProtoMessage() {} func (x *NotCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[48] + mi := &file_deephaven_proto_table_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4711,7 +4796,7 @@ func (x *NotCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use NotCondition.ProtoReflect.Descriptor instead. func (*NotCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{48} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{49} } func (x *NotCondition) GetFilter() *Condition { @@ -4735,7 +4820,7 @@ type CompareCondition struct { func (x *CompareCondition) Reset() { *x = CompareCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[49] + mi := &file_deephaven_proto_table_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4748,7 +4833,7 @@ func (x *CompareCondition) String() string { func (*CompareCondition) ProtoMessage() {} func (x *CompareCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[49] + mi := &file_deephaven_proto_table_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4761,7 +4846,7 @@ func (x *CompareCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use CompareCondition.ProtoReflect.Descriptor instead. func (*CompareCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{49} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{50} } func (x *CompareCondition) GetOperation() CompareCondition_CompareOperation { @@ -4806,7 +4891,7 @@ type InCondition struct { func (x *InCondition) Reset() { *x = InCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[50] + mi := &file_deephaven_proto_table_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4819,7 +4904,7 @@ func (x *InCondition) String() string { func (*InCondition) ProtoMessage() {} func (x *InCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[50] + mi := &file_deephaven_proto_table_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4832,7 +4917,7 @@ func (x *InCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use InCondition.ProtoReflect.Descriptor instead. func (*InCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{50} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{51} } func (x *InCondition) GetTarget() *Value { @@ -4876,7 +4961,7 @@ type InvokeCondition struct { func (x *InvokeCondition) Reset() { *x = InvokeCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[51] + mi := &file_deephaven_proto_table_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4889,7 +4974,7 @@ func (x *InvokeCondition) String() string { func (*InvokeCondition) ProtoMessage() {} func (x *InvokeCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[51] + mi := &file_deephaven_proto_table_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4902,7 +4987,7 @@ func (x *InvokeCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use InvokeCondition.ProtoReflect.Descriptor instead. func (*InvokeCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{51} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{52} } func (x *InvokeCondition) GetMethod() string { @@ -4937,7 +5022,7 @@ type IsNullCondition struct { func (x *IsNullCondition) Reset() { *x = IsNullCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[52] + mi := &file_deephaven_proto_table_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4950,7 +5035,7 @@ func (x *IsNullCondition) String() string { func (*IsNullCondition) ProtoMessage() {} func (x *IsNullCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[52] + mi := &file_deephaven_proto_table_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4963,7 +5048,7 @@ func (x *IsNullCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use IsNullCondition.ProtoReflect.Descriptor instead. func (*IsNullCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{52} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{53} } func (x *IsNullCondition) GetReference() *Reference { @@ -4987,7 +5072,7 @@ type MatchesCondition struct { func (x *MatchesCondition) Reset() { *x = MatchesCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[53] + mi := &file_deephaven_proto_table_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5000,7 +5085,7 @@ func (x *MatchesCondition) String() string { func (*MatchesCondition) ProtoMessage() {} func (x *MatchesCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[53] + mi := &file_deephaven_proto_table_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5013,7 +5098,7 @@ func (x *MatchesCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use MatchesCondition.ProtoReflect.Descriptor instead. func (*MatchesCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{53} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{54} } func (x *MatchesCondition) GetReference() *Reference { @@ -5058,7 +5143,7 @@ type ContainsCondition struct { func (x *ContainsCondition) Reset() { *x = ContainsCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[54] + mi := &file_deephaven_proto_table_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5071,7 +5156,7 @@ func (x *ContainsCondition) String() string { func (*ContainsCondition) ProtoMessage() {} func (x *ContainsCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[54] + mi := &file_deephaven_proto_table_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5084,7 +5169,7 @@ func (x *ContainsCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use ContainsCondition.ProtoReflect.Descriptor instead. func (*ContainsCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{54} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{55} } func (x *ContainsCondition) GetReference() *Reference { @@ -5128,7 +5213,7 @@ type SearchCondition struct { func (x *SearchCondition) Reset() { *x = SearchCondition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[55] + mi := &file_deephaven_proto_table_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5141,7 +5226,7 @@ func (x *SearchCondition) String() string { func (*SearchCondition) ProtoMessage() {} func (x *SearchCondition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[55] + mi := &file_deephaven_proto_table_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5154,7 +5239,7 @@ func (x *SearchCondition) ProtoReflect() protoreflect.Message { // Deprecated: Use SearchCondition.ProtoReflect.Descriptor instead. func (*SearchCondition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{55} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{56} } func (x *SearchCondition) GetSearchString() string { @@ -5183,7 +5268,7 @@ type FlattenRequest struct { func (x *FlattenRequest) Reset() { *x = FlattenRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[56] + mi := &file_deephaven_proto_table_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5196,7 +5281,7 @@ func (x *FlattenRequest) String() string { func (*FlattenRequest) ProtoMessage() {} func (x *FlattenRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[56] + mi := &file_deephaven_proto_table_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5209,7 +5294,7 @@ func (x *FlattenRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use FlattenRequest.ProtoReflect.Descriptor instead. func (*FlattenRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{56} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{57} } func (x *FlattenRequest) GetResultId() *ticket.Ticket { @@ -5238,7 +5323,7 @@ type MetaTableRequest struct { func (x *MetaTableRequest) Reset() { *x = MetaTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[57] + mi := &file_deephaven_proto_table_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5251,7 +5336,7 @@ func (x *MetaTableRequest) String() string { func (*MetaTableRequest) ProtoMessage() {} func (x *MetaTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[57] + mi := &file_deephaven_proto_table_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5264,7 +5349,7 @@ func (x *MetaTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use MetaTableRequest.ProtoReflect.Descriptor instead. func (*MetaTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{57} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{58} } func (x *MetaTableRequest) GetResultId() *ticket.Ticket { @@ -5297,7 +5382,7 @@ type RunChartDownsampleRequest struct { func (x *RunChartDownsampleRequest) Reset() { *x = RunChartDownsampleRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[58] + mi := &file_deephaven_proto_table_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5310,7 +5395,7 @@ func (x *RunChartDownsampleRequest) String() string { func (*RunChartDownsampleRequest) ProtoMessage() {} func (x *RunChartDownsampleRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[58] + mi := &file_deephaven_proto_table_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5323,7 +5408,7 @@ func (x *RunChartDownsampleRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RunChartDownsampleRequest.ProtoReflect.Descriptor instead. func (*RunChartDownsampleRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{58} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59} } func (x *RunChartDownsampleRequest) GetResultId() *ticket.Ticket { @@ -5385,7 +5470,7 @@ type CreateInputTableRequest struct { func (x *CreateInputTableRequest) Reset() { *x = CreateInputTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[59] + mi := &file_deephaven_proto_table_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5398,7 +5483,7 @@ func (x *CreateInputTableRequest) String() string { func (*CreateInputTableRequest) ProtoMessage() {} func (x *CreateInputTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[59] + mi := &file_deephaven_proto_table_proto_msgTypes[60] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5411,7 +5496,7 @@ func (x *CreateInputTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CreateInputTableRequest.ProtoReflect.Descriptor instead. func (*CreateInputTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{60} } func (x *CreateInputTableRequest) GetResultId() *ticket.Ticket { @@ -5483,7 +5568,7 @@ type WhereInRequest struct { func (x *WhereInRequest) Reset() { *x = WhereInRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[60] + mi := &file_deephaven_proto_table_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5496,7 +5581,7 @@ func (x *WhereInRequest) String() string { func (*WhereInRequest) ProtoMessage() {} func (x *WhereInRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[60] + mi := &file_deephaven_proto_table_proto_msgTypes[61] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5509,7 +5594,7 @@ func (x *WhereInRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use WhereInRequest.ProtoReflect.Descriptor instead. func (*WhereInRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{60} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{61} } func (x *WhereInRequest) GetResultId() *ticket.Ticket { @@ -5564,7 +5649,7 @@ type ColumnStatisticsRequest struct { func (x *ColumnStatisticsRequest) Reset() { *x = ColumnStatisticsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[61] + mi := &file_deephaven_proto_table_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5577,7 +5662,7 @@ func (x *ColumnStatisticsRequest) String() string { func (*ColumnStatisticsRequest) ProtoMessage() {} func (x *ColumnStatisticsRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[61] + mi := &file_deephaven_proto_table_proto_msgTypes[62] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5590,7 +5675,7 @@ func (x *ColumnStatisticsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ColumnStatisticsRequest.ProtoReflect.Descriptor instead. func (*ColumnStatisticsRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{61} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{62} } func (x *ColumnStatisticsRequest) GetResultId() *ticket.Ticket { @@ -5635,7 +5720,7 @@ type SliceRequest struct { func (x *SliceRequest) Reset() { *x = SliceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[62] + mi := &file_deephaven_proto_table_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5648,7 +5733,7 @@ func (x *SliceRequest) String() string { func (*SliceRequest) ProtoMessage() {} func (x *SliceRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[62] + mi := &file_deephaven_proto_table_proto_msgTypes[63] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5661,7 +5746,7 @@ func (x *SliceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SliceRequest.ProtoReflect.Descriptor instead. func (*SliceRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{62} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{63} } func (x *SliceRequest) GetResultId() *ticket.Ticket { @@ -5703,7 +5788,7 @@ type BatchTableRequest struct { func (x *BatchTableRequest) Reset() { *x = BatchTableRequest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[63] + mi := &file_deephaven_proto_table_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5716,7 +5801,7 @@ func (x *BatchTableRequest) String() string { func (*BatchTableRequest) ProtoMessage() {} func (x *BatchTableRequest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[63] + mi := &file_deephaven_proto_table_proto_msgTypes[64] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5729,7 +5814,7 @@ func (x *BatchTableRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BatchTableRequest.ProtoReflect.Descriptor instead. func (*BatchTableRequest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{63} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{64} } func (x *BatchTableRequest) GetOps() []*BatchTableRequest_Operation { @@ -5750,7 +5835,7 @@ type UpdateByWindowScale_UpdateByWindowTicks struct { func (x *UpdateByWindowScale_UpdateByWindowTicks) Reset() { *x = UpdateByWindowScale_UpdateByWindowTicks{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[64] + mi := &file_deephaven_proto_table_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5763,7 +5848,7 @@ func (x *UpdateByWindowScale_UpdateByWindowTicks) String() string { func (*UpdateByWindowScale_UpdateByWindowTicks) ProtoMessage() {} func (x *UpdateByWindowScale_UpdateByWindowTicks) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[64] + mi := &file_deephaven_proto_table_proto_msgTypes[65] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5776,7 +5861,7 @@ func (x *UpdateByWindowScale_UpdateByWindowTicks) ProtoReflect() protoreflect.Me // Deprecated: Use UpdateByWindowScale_UpdateByWindowTicks.ProtoReflect.Descriptor instead. func (*UpdateByWindowScale_UpdateByWindowTicks) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{10, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{11, 0} } func (x *UpdateByWindowScale_UpdateByWindowTicks) GetTicks() float64 { @@ -5801,7 +5886,7 @@ type UpdateByWindowScale_UpdateByWindowTime struct { func (x *UpdateByWindowScale_UpdateByWindowTime) Reset() { *x = UpdateByWindowScale_UpdateByWindowTime{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[65] + mi := &file_deephaven_proto_table_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5814,7 +5899,7 @@ func (x *UpdateByWindowScale_UpdateByWindowTime) String() string { func (*UpdateByWindowScale_UpdateByWindowTime) ProtoMessage() {} func (x *UpdateByWindowScale_UpdateByWindowTime) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[65] + mi := &file_deephaven_proto_table_proto_msgTypes[66] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5827,7 +5912,7 @@ func (x *UpdateByWindowScale_UpdateByWindowTime) ProtoReflect() protoreflect.Mes // Deprecated: Use UpdateByWindowScale_UpdateByWindowTime.ProtoReflect.Descriptor instead. func (*UpdateByWindowScale_UpdateByWindowTime) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{10, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{11, 1} } func (x *UpdateByWindowScale_UpdateByWindowTime) GetColumn() string { @@ -5908,7 +5993,7 @@ type UpdateByRequest_UpdateByOptions struct { func (x *UpdateByRequest_UpdateByOptions) Reset() { *x = UpdateByRequest_UpdateByOptions{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[66] + mi := &file_deephaven_proto_table_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5921,7 +6006,7 @@ func (x *UpdateByRequest_UpdateByOptions) String() string { func (*UpdateByRequest_UpdateByOptions) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOptions) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[66] + mi := &file_deephaven_proto_table_proto_msgTypes[67] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5934,7 +6019,7 @@ func (x *UpdateByRequest_UpdateByOptions) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateByRequest_UpdateByOptions.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOptions) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 0} } func (x *UpdateByRequest_UpdateByOptions) GetUseRedirection() bool { @@ -5999,7 +6084,7 @@ type UpdateByRequest_UpdateByOperation struct { func (x *UpdateByRequest_UpdateByOperation) Reset() { *x = UpdateByRequest_UpdateByOperation{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[67] + mi := &file_deephaven_proto_table_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6012,7 +6097,7 @@ func (x *UpdateByRequest_UpdateByOperation) String() string { func (*UpdateByRequest_UpdateByOperation) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[67] + mi := &file_deephaven_proto_table_proto_msgTypes[68] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6025,7 +6110,7 @@ func (x *UpdateByRequest_UpdateByOperation) ProtoReflect() protoreflect.Message // Deprecated: Use UpdateByRequest_UpdateByOperation.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1} } func (m *UpdateByRequest_UpdateByOperation) GetType() isUpdateByRequest_UpdateByOperation_Type { @@ -6064,7 +6149,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn struct { func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[68] + mi := &file_deephaven_proto_table_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6077,7 +6162,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn) String() string { func (*UpdateByRequest_UpdateByOperation_UpdateByColumn) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[68] + mi := &file_deephaven_proto_table_proto_msgTypes[69] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6090,7 +6175,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn) ProtoReflect() protor // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn) GetSpec() *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec { @@ -6140,7 +6225,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec struct { func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[69] + mi := &file_deephaven_proto_table_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6153,7 +6238,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) String() func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[69] + mi := &file_deephaven_proto_table_proto_msgTypes[70] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6166,7 +6251,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) ProtoRef // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0} } func (m *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) GetType() isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type { @@ -6483,7 +6568,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumul func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[70] + mi := &file_deephaven_proto_table_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6497,7 +6582,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCum } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[70] + mi := &file_deephaven_proto_table_proto_msgTypes[71] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6510,7 +6595,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByC // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 0} } type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin struct { @@ -6522,7 +6607,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumul func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[71] + mi := &file_deephaven_proto_table_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6536,7 +6621,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCum } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[71] + mi := &file_deephaven_proto_table_proto_msgTypes[72] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6549,7 +6634,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByC // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 1} } type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax struct { @@ -6561,7 +6646,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumul func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[72] + mi := &file_deephaven_proto_table_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6575,7 +6660,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCum } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[72] + mi := &file_deephaven_proto_table_proto_msgTypes[73] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6588,7 +6673,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByC // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 2} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 2} } type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct struct { @@ -6600,7 +6685,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumul func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[73] + mi := &file_deephaven_proto_table_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6614,7 +6699,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCum } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[73] + mi := &file_deephaven_proto_table_proto_msgTypes[74] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6627,7 +6712,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByC // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 3} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 3} } type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill struct { @@ -6639,7 +6724,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[74] + mi := &file_deephaven_proto_table_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6652,7 +6737,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByF func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[74] + mi := &file_deephaven_proto_table_proto_msgTypes[75] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6665,7 +6750,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByF // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 4} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 4} } type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma struct { @@ -6680,7 +6765,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma s func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[75] + mi := &file_deephaven_proto_table_proto_msgTypes[76] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6693,7 +6778,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[75] + mi := &file_deephaven_proto_table_proto_msgTypes[76] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6706,7 +6791,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 5} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 5} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma) GetOptions() *UpdateByEmOptions { @@ -6735,7 +6820,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms s func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[76] + mi := &file_deephaven_proto_table_proto_msgTypes[77] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6748,7 +6833,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[76] + mi := &file_deephaven_proto_table_proto_msgTypes[77] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6761,7 +6846,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 6} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 6} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms) GetOptions() *UpdateByEmOptions { @@ -6790,7 +6875,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[77] + mi := &file_deephaven_proto_table_proto_msgTypes[78] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6803,7 +6888,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[77] + mi := &file_deephaven_proto_table_proto_msgTypes[78] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6816,7 +6901,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 7} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 7} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin) GetOptions() *UpdateByEmOptions { @@ -6845,7 +6930,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[78] + mi := &file_deephaven_proto_table_proto_msgTypes[79] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6858,7 +6943,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[78] + mi := &file_deephaven_proto_table_proto_msgTypes[79] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6871,7 +6956,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 8} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 8} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax) GetOptions() *UpdateByEmOptions { @@ -6900,7 +6985,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[79] + mi := &file_deephaven_proto_table_proto_msgTypes[80] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6913,7 +6998,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[79] + mi := &file_deephaven_proto_table_proto_msgTypes[80] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6926,7 +7011,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByE // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 9} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 9} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd) GetOptions() *UpdateByEmOptions { @@ -6954,7 +7039,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[80] + mi := &file_deephaven_proto_table_proto_msgTypes[81] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6967,7 +7052,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByD func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta) ProtoMessage() {} func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[80] + mi := &file_deephaven_proto_table_proto_msgTypes[81] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6980,7 +7065,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByD // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 10} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 10} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta) GetOptions() *UpdateByDeltaOptions { @@ -7002,7 +7087,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[81] + mi := &file_deephaven_proto_table_proto_msgTypes[82] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7016,7 +7101,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[81] + mi := &file_deephaven_proto_table_proto_msgTypes[82] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7029,7 +7114,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 11} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 11} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum) GetReverseWindowScale() *UpdateByWindowScale { @@ -7058,7 +7143,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[82] + mi := &file_deephaven_proto_table_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7072,7 +7157,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[82] + mi := &file_deephaven_proto_table_proto_msgTypes[83] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7085,7 +7170,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 12} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 12} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup) GetReverseWindowScale() *UpdateByWindowScale { @@ -7114,7 +7199,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[83] + mi := &file_deephaven_proto_table_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7128,7 +7213,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[83] + mi := &file_deephaven_proto_table_proto_msgTypes[84] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7141,7 +7226,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 13} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 13} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg) GetReverseWindowScale() *UpdateByWindowScale { @@ -7170,7 +7255,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[84] + mi := &file_deephaven_proto_table_proto_msgTypes[85] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7184,7 +7269,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[84] + mi := &file_deephaven_proto_table_proto_msgTypes[85] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7197,7 +7282,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 14} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 14} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin) GetReverseWindowScale() *UpdateByWindowScale { @@ -7226,7 +7311,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[85] + mi := &file_deephaven_proto_table_proto_msgTypes[86] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7240,7 +7325,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[85] + mi := &file_deephaven_proto_table_proto_msgTypes[86] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7253,7 +7338,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 15} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 15} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax) GetReverseWindowScale() *UpdateByWindowScale { @@ -7282,7 +7367,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[86] + mi := &file_deephaven_proto_table_proto_msgTypes[87] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7296,7 +7381,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[86] + mi := &file_deephaven_proto_table_proto_msgTypes[87] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7309,7 +7394,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 16} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 16} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct) GetReverseWindowScale() *UpdateByWindowScale { @@ -7338,7 +7423,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[87] + mi := &file_deephaven_proto_table_proto_msgTypes[88] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7352,7 +7437,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[87] + mi := &file_deephaven_proto_table_proto_msgTypes[88] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7365,7 +7450,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 17} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 17} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount) GetReverseWindowScale() *UpdateByWindowScale { @@ -7394,7 +7479,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[88] + mi := &file_deephaven_proto_table_proto_msgTypes[89] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7408,7 +7493,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[88] + mi := &file_deephaven_proto_table_proto_msgTypes[89] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7421,7 +7506,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 18} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 18} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd) GetReverseWindowScale() *UpdateByWindowScale { @@ -7452,7 +7537,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[89] + mi := &file_deephaven_proto_table_proto_msgTypes[90] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7466,7 +7551,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[89] + mi := &file_deephaven_proto_table_proto_msgTypes[90] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7479,7 +7564,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 19} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 19} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg) GetReverseWindowScale() *UpdateByWindowScale { @@ -7517,7 +7602,7 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRolli func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula) Reset() { *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[90] + mi := &file_deephaven_proto_table_proto_msgTypes[91] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7531,7 +7616,7 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRol } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[90] + mi := &file_deephaven_proto_table_proto_msgTypes[91] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7544,7 +7629,7 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR // Deprecated: Use UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula.ProtoReflect.Descriptor instead. func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{13, 1, 0, 0, 20} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 20} } func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula) GetReverseWindowScale() *UpdateByWindowScale { @@ -7590,7 +7675,7 @@ type ComboAggregateRequest_Aggregate struct { func (x *ComboAggregateRequest_Aggregate) Reset() { *x = ComboAggregateRequest_Aggregate{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[91] + mi := &file_deephaven_proto_table_proto_msgTypes[92] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7603,7 +7688,7 @@ func (x *ComboAggregateRequest_Aggregate) String() string { func (*ComboAggregateRequest_Aggregate) ProtoMessage() {} func (x *ComboAggregateRequest_Aggregate) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[91] + mi := &file_deephaven_proto_table_proto_msgTypes[92] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7616,7 +7701,7 @@ func (x *ComboAggregateRequest_Aggregate) ProtoReflect() protoreflect.Message { // Deprecated: Use ComboAggregateRequest_Aggregate.ProtoReflect.Descriptor instead. func (*ComboAggregateRequest_Aggregate) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{32, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{33, 0} } func (x *ComboAggregateRequest_Aggregate) GetType() ComboAggregateRequest_AggType { @@ -7669,7 +7754,7 @@ type AggSpec_AggSpecApproximatePercentile struct { func (x *AggSpec_AggSpecApproximatePercentile) Reset() { *x = AggSpec_AggSpecApproximatePercentile{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[92] + mi := &file_deephaven_proto_table_proto_msgTypes[93] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7682,7 +7767,7 @@ func (x *AggSpec_AggSpecApproximatePercentile) String() string { func (*AggSpec_AggSpecApproximatePercentile) ProtoMessage() {} func (x *AggSpec_AggSpecApproximatePercentile) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[92] + mi := &file_deephaven_proto_table_proto_msgTypes[93] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7695,7 +7780,7 @@ func (x *AggSpec_AggSpecApproximatePercentile) ProtoReflect() protoreflect.Messa // Deprecated: Use AggSpec_AggSpecApproximatePercentile.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecApproximatePercentile) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 0} } func (x *AggSpec_AggSpecApproximatePercentile) GetPercentile() float64 { @@ -7724,7 +7809,7 @@ type AggSpec_AggSpecCountDistinct struct { func (x *AggSpec_AggSpecCountDistinct) Reset() { *x = AggSpec_AggSpecCountDistinct{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[93] + mi := &file_deephaven_proto_table_proto_msgTypes[94] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7737,7 +7822,7 @@ func (x *AggSpec_AggSpecCountDistinct) String() string { func (*AggSpec_AggSpecCountDistinct) ProtoMessage() {} func (x *AggSpec_AggSpecCountDistinct) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[93] + mi := &file_deephaven_proto_table_proto_msgTypes[94] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7750,7 +7835,7 @@ func (x *AggSpec_AggSpecCountDistinct) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecCountDistinct.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecCountDistinct) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 1} } func (x *AggSpec_AggSpecCountDistinct) GetCountNulls() bool { @@ -7772,7 +7857,7 @@ type AggSpec_AggSpecDistinct struct { func (x *AggSpec_AggSpecDistinct) Reset() { *x = AggSpec_AggSpecDistinct{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[94] + mi := &file_deephaven_proto_table_proto_msgTypes[95] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7785,7 +7870,7 @@ func (x *AggSpec_AggSpecDistinct) String() string { func (*AggSpec_AggSpecDistinct) ProtoMessage() {} func (x *AggSpec_AggSpecDistinct) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[94] + mi := &file_deephaven_proto_table_proto_msgTypes[95] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7798,7 +7883,7 @@ func (x *AggSpec_AggSpecDistinct) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecDistinct.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecDistinct) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 2} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 2} } func (x *AggSpec_AggSpecDistinct) GetIncludeNulls() bool { @@ -7822,7 +7907,7 @@ type AggSpec_AggSpecFormula struct { func (x *AggSpec_AggSpecFormula) Reset() { *x = AggSpec_AggSpecFormula{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[95] + mi := &file_deephaven_proto_table_proto_msgTypes[96] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7835,7 +7920,7 @@ func (x *AggSpec_AggSpecFormula) String() string { func (*AggSpec_AggSpecFormula) ProtoMessage() {} func (x *AggSpec_AggSpecFormula) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[95] + mi := &file_deephaven_proto_table_proto_msgTypes[96] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7848,7 +7933,7 @@ func (x *AggSpec_AggSpecFormula) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecFormula.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecFormula) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 3} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 3} } func (x *AggSpec_AggSpecFormula) GetFormula() string { @@ -7878,7 +7963,7 @@ type AggSpec_AggSpecMedian struct { func (x *AggSpec_AggSpecMedian) Reset() { *x = AggSpec_AggSpecMedian{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[96] + mi := &file_deephaven_proto_table_proto_msgTypes[97] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7891,7 +7976,7 @@ func (x *AggSpec_AggSpecMedian) String() string { func (*AggSpec_AggSpecMedian) ProtoMessage() {} func (x *AggSpec_AggSpecMedian) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[96] + mi := &file_deephaven_proto_table_proto_msgTypes[97] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7904,7 +7989,7 @@ func (x *AggSpec_AggSpecMedian) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecMedian.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecMedian) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 4} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 4} } func (x *AggSpec_AggSpecMedian) GetAverageEvenlyDivided() bool { @@ -7929,7 +8014,7 @@ type AggSpec_AggSpecPercentile struct { func (x *AggSpec_AggSpecPercentile) Reset() { *x = AggSpec_AggSpecPercentile{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[97] + mi := &file_deephaven_proto_table_proto_msgTypes[98] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7942,7 +8027,7 @@ func (x *AggSpec_AggSpecPercentile) String() string { func (*AggSpec_AggSpecPercentile) ProtoMessage() {} func (x *AggSpec_AggSpecPercentile) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[97] + mi := &file_deephaven_proto_table_proto_msgTypes[98] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7955,7 +8040,7 @@ func (x *AggSpec_AggSpecPercentile) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecPercentile.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecPercentile) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 5} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 5} } func (x *AggSpec_AggSpecPercentile) GetPercentile() float64 { @@ -7984,7 +8069,7 @@ type AggSpec_AggSpecSorted struct { func (x *AggSpec_AggSpecSorted) Reset() { *x = AggSpec_AggSpecSorted{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[98] + mi := &file_deephaven_proto_table_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7997,7 +8082,7 @@ func (x *AggSpec_AggSpecSorted) String() string { func (*AggSpec_AggSpecSorted) ProtoMessage() {} func (x *AggSpec_AggSpecSorted) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[98] + mi := &file_deephaven_proto_table_proto_msgTypes[99] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8010,7 +8095,7 @@ func (x *AggSpec_AggSpecSorted) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecSorted.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecSorted) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 6} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 6} } func (x *AggSpec_AggSpecSorted) GetColumns() []*AggSpec_AggSpecSortedColumn { @@ -8032,7 +8117,7 @@ type AggSpec_AggSpecSortedColumn struct { func (x *AggSpec_AggSpecSortedColumn) Reset() { *x = AggSpec_AggSpecSortedColumn{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[99] + mi := &file_deephaven_proto_table_proto_msgTypes[100] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8045,7 +8130,7 @@ func (x *AggSpec_AggSpecSortedColumn) String() string { func (*AggSpec_AggSpecSortedColumn) ProtoMessage() {} func (x *AggSpec_AggSpecSortedColumn) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[99] + mi := &file_deephaven_proto_table_proto_msgTypes[100] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8058,7 +8143,7 @@ func (x *AggSpec_AggSpecSortedColumn) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecSortedColumn.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecSortedColumn) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 7} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 7} } func (x *AggSpec_AggSpecSortedColumn) GetColumnName() string { @@ -8081,7 +8166,7 @@ type AggSpec_AggSpecTDigest struct { func (x *AggSpec_AggSpecTDigest) Reset() { *x = AggSpec_AggSpecTDigest{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[100] + mi := &file_deephaven_proto_table_proto_msgTypes[101] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8094,7 +8179,7 @@ func (x *AggSpec_AggSpecTDigest) String() string { func (*AggSpec_AggSpecTDigest) ProtoMessage() {} func (x *AggSpec_AggSpecTDigest) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[100] + mi := &file_deephaven_proto_table_proto_msgTypes[101] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8107,7 +8192,7 @@ func (x *AggSpec_AggSpecTDigest) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecTDigest.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecTDigest) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 8} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 8} } func (x *AggSpec_AggSpecTDigest) GetCompression() float64 { @@ -8131,7 +8216,7 @@ type AggSpec_AggSpecUnique struct { func (x *AggSpec_AggSpecUnique) Reset() { *x = AggSpec_AggSpecUnique{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[101] + mi := &file_deephaven_proto_table_proto_msgTypes[102] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8144,7 +8229,7 @@ func (x *AggSpec_AggSpecUnique) String() string { func (*AggSpec_AggSpecUnique) ProtoMessage() {} func (x *AggSpec_AggSpecUnique) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[101] + mi := &file_deephaven_proto_table_proto_msgTypes[102] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8157,7 +8242,7 @@ func (x *AggSpec_AggSpecUnique) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecUnique.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecUnique) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 9} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 9} } func (x *AggSpec_AggSpecUnique) GetIncludeNulls() bool { @@ -8196,7 +8281,7 @@ type AggSpec_AggSpecNonUniqueSentinel struct { func (x *AggSpec_AggSpecNonUniqueSentinel) Reset() { *x = AggSpec_AggSpecNonUniqueSentinel{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[102] + mi := &file_deephaven_proto_table_proto_msgTypes[103] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8209,7 +8294,7 @@ func (x *AggSpec_AggSpecNonUniqueSentinel) String() string { func (*AggSpec_AggSpecNonUniqueSentinel) ProtoMessage() {} func (x *AggSpec_AggSpecNonUniqueSentinel) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[102] + mi := &file_deephaven_proto_table_proto_msgTypes[103] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8222,7 +8307,7 @@ func (x *AggSpec_AggSpecNonUniqueSentinel) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecNonUniqueSentinel.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecNonUniqueSentinel) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 10} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 10} } func (m *AggSpec_AggSpecNonUniqueSentinel) GetType() isAggSpec_AggSpecNonUniqueSentinel_Type { @@ -8381,7 +8466,7 @@ type AggSpec_AggSpecWeighted struct { func (x *AggSpec_AggSpecWeighted) Reset() { *x = AggSpec_AggSpecWeighted{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[103] + mi := &file_deephaven_proto_table_proto_msgTypes[104] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8394,7 +8479,7 @@ func (x *AggSpec_AggSpecWeighted) String() string { func (*AggSpec_AggSpecWeighted) ProtoMessage() {} func (x *AggSpec_AggSpecWeighted) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[103] + mi := &file_deephaven_proto_table_proto_msgTypes[104] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8407,7 +8492,7 @@ func (x *AggSpec_AggSpecWeighted) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecWeighted.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecWeighted) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 11} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 11} } func (x *AggSpec_AggSpecWeighted) GetWeightColumn() string { @@ -8426,7 +8511,7 @@ type AggSpec_AggSpecAbsSum struct { func (x *AggSpec_AggSpecAbsSum) Reset() { *x = AggSpec_AggSpecAbsSum{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[104] + mi := &file_deephaven_proto_table_proto_msgTypes[105] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8439,7 +8524,7 @@ func (x *AggSpec_AggSpecAbsSum) String() string { func (*AggSpec_AggSpecAbsSum) ProtoMessage() {} func (x *AggSpec_AggSpecAbsSum) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[104] + mi := &file_deephaven_proto_table_proto_msgTypes[105] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8452,7 +8537,7 @@ func (x *AggSpec_AggSpecAbsSum) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecAbsSum.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecAbsSum) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 12} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 12} } type AggSpec_AggSpecAvg struct { @@ -8464,7 +8549,7 @@ type AggSpec_AggSpecAvg struct { func (x *AggSpec_AggSpecAvg) Reset() { *x = AggSpec_AggSpecAvg{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[105] + mi := &file_deephaven_proto_table_proto_msgTypes[106] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8477,7 +8562,7 @@ func (x *AggSpec_AggSpecAvg) String() string { func (*AggSpec_AggSpecAvg) ProtoMessage() {} func (x *AggSpec_AggSpecAvg) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[105] + mi := &file_deephaven_proto_table_proto_msgTypes[106] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8490,7 +8575,7 @@ func (x *AggSpec_AggSpecAvg) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecAvg.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecAvg) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 13} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 13} } type AggSpec_AggSpecFirst struct { @@ -8502,7 +8587,7 @@ type AggSpec_AggSpecFirst struct { func (x *AggSpec_AggSpecFirst) Reset() { *x = AggSpec_AggSpecFirst{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[106] + mi := &file_deephaven_proto_table_proto_msgTypes[107] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8515,7 +8600,7 @@ func (x *AggSpec_AggSpecFirst) String() string { func (*AggSpec_AggSpecFirst) ProtoMessage() {} func (x *AggSpec_AggSpecFirst) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[106] + mi := &file_deephaven_proto_table_proto_msgTypes[107] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8528,7 +8613,7 @@ func (x *AggSpec_AggSpecFirst) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecFirst.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecFirst) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 14} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 14} } type AggSpec_AggSpecFreeze struct { @@ -8540,7 +8625,7 @@ type AggSpec_AggSpecFreeze struct { func (x *AggSpec_AggSpecFreeze) Reset() { *x = AggSpec_AggSpecFreeze{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[107] + mi := &file_deephaven_proto_table_proto_msgTypes[108] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8553,7 +8638,7 @@ func (x *AggSpec_AggSpecFreeze) String() string { func (*AggSpec_AggSpecFreeze) ProtoMessage() {} func (x *AggSpec_AggSpecFreeze) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[107] + mi := &file_deephaven_proto_table_proto_msgTypes[108] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8566,7 +8651,7 @@ func (x *AggSpec_AggSpecFreeze) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecFreeze.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecFreeze) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 15} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 15} } type AggSpec_AggSpecGroup struct { @@ -8578,7 +8663,7 @@ type AggSpec_AggSpecGroup struct { func (x *AggSpec_AggSpecGroup) Reset() { *x = AggSpec_AggSpecGroup{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[108] + mi := &file_deephaven_proto_table_proto_msgTypes[109] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8591,7 +8676,7 @@ func (x *AggSpec_AggSpecGroup) String() string { func (*AggSpec_AggSpecGroup) ProtoMessage() {} func (x *AggSpec_AggSpecGroup) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[108] + mi := &file_deephaven_proto_table_proto_msgTypes[109] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8604,7 +8689,7 @@ func (x *AggSpec_AggSpecGroup) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecGroup.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecGroup) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 16} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 16} } type AggSpec_AggSpecLast struct { @@ -8616,7 +8701,7 @@ type AggSpec_AggSpecLast struct { func (x *AggSpec_AggSpecLast) Reset() { *x = AggSpec_AggSpecLast{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[109] + mi := &file_deephaven_proto_table_proto_msgTypes[110] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8629,7 +8714,7 @@ func (x *AggSpec_AggSpecLast) String() string { func (*AggSpec_AggSpecLast) ProtoMessage() {} func (x *AggSpec_AggSpecLast) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[109] + mi := &file_deephaven_proto_table_proto_msgTypes[110] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8642,7 +8727,7 @@ func (x *AggSpec_AggSpecLast) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecLast.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecLast) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 17} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 17} } type AggSpec_AggSpecMax struct { @@ -8654,7 +8739,7 @@ type AggSpec_AggSpecMax struct { func (x *AggSpec_AggSpecMax) Reset() { *x = AggSpec_AggSpecMax{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[110] + mi := &file_deephaven_proto_table_proto_msgTypes[111] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8667,7 +8752,7 @@ func (x *AggSpec_AggSpecMax) String() string { func (*AggSpec_AggSpecMax) ProtoMessage() {} func (x *AggSpec_AggSpecMax) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[110] + mi := &file_deephaven_proto_table_proto_msgTypes[111] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8680,7 +8765,7 @@ func (x *AggSpec_AggSpecMax) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecMax.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecMax) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 18} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 18} } type AggSpec_AggSpecMin struct { @@ -8692,7 +8777,7 @@ type AggSpec_AggSpecMin struct { func (x *AggSpec_AggSpecMin) Reset() { *x = AggSpec_AggSpecMin{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[111] + mi := &file_deephaven_proto_table_proto_msgTypes[112] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8705,7 +8790,7 @@ func (x *AggSpec_AggSpecMin) String() string { func (*AggSpec_AggSpecMin) ProtoMessage() {} func (x *AggSpec_AggSpecMin) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[111] + mi := &file_deephaven_proto_table_proto_msgTypes[112] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8718,7 +8803,7 @@ func (x *AggSpec_AggSpecMin) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecMin.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecMin) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 19} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 19} } type AggSpec_AggSpecStd struct { @@ -8730,7 +8815,7 @@ type AggSpec_AggSpecStd struct { func (x *AggSpec_AggSpecStd) Reset() { *x = AggSpec_AggSpecStd{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[112] + mi := &file_deephaven_proto_table_proto_msgTypes[113] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8743,7 +8828,7 @@ func (x *AggSpec_AggSpecStd) String() string { func (*AggSpec_AggSpecStd) ProtoMessage() {} func (x *AggSpec_AggSpecStd) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[112] + mi := &file_deephaven_proto_table_proto_msgTypes[113] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8756,7 +8841,7 @@ func (x *AggSpec_AggSpecStd) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecStd.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecStd) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 20} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 20} } type AggSpec_AggSpecSum struct { @@ -8768,7 +8853,7 @@ type AggSpec_AggSpecSum struct { func (x *AggSpec_AggSpecSum) Reset() { *x = AggSpec_AggSpecSum{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[113] + mi := &file_deephaven_proto_table_proto_msgTypes[114] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8781,7 +8866,7 @@ func (x *AggSpec_AggSpecSum) String() string { func (*AggSpec_AggSpecSum) ProtoMessage() {} func (x *AggSpec_AggSpecSum) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[113] + mi := &file_deephaven_proto_table_proto_msgTypes[114] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8794,7 +8879,7 @@ func (x *AggSpec_AggSpecSum) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecSum.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecSum) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 21} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 21} } type AggSpec_AggSpecVar struct { @@ -8806,7 +8891,7 @@ type AggSpec_AggSpecVar struct { func (x *AggSpec_AggSpecVar) Reset() { *x = AggSpec_AggSpecVar{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[114] + mi := &file_deephaven_proto_table_proto_msgTypes[115] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8819,7 +8904,7 @@ func (x *AggSpec_AggSpecVar) String() string { func (*AggSpec_AggSpecVar) ProtoMessage() {} func (x *AggSpec_AggSpecVar) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[114] + mi := &file_deephaven_proto_table_proto_msgTypes[115] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8832,7 +8917,7 @@ func (x *AggSpec_AggSpecVar) ProtoReflect() protoreflect.Message { // Deprecated: Use AggSpec_AggSpecVar.ProtoReflect.Descriptor instead. func (*AggSpec_AggSpecVar) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{34, 22} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{35, 22} } type Aggregation_AggregationColumns struct { @@ -8847,7 +8932,7 @@ type Aggregation_AggregationColumns struct { func (x *Aggregation_AggregationColumns) Reset() { *x = Aggregation_AggregationColumns{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[115] + mi := &file_deephaven_proto_table_proto_msgTypes[116] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8860,7 +8945,7 @@ func (x *Aggregation_AggregationColumns) String() string { func (*Aggregation_AggregationColumns) ProtoMessage() {} func (x *Aggregation_AggregationColumns) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[115] + mi := &file_deephaven_proto_table_proto_msgTypes[116] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8873,7 +8958,7 @@ func (x *Aggregation_AggregationColumns) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationColumns.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationColumns) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{36, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37, 0} } func (x *Aggregation_AggregationColumns) GetSpec() *AggSpec { @@ -8902,7 +8987,7 @@ type Aggregation_AggregationCount struct { func (x *Aggregation_AggregationCount) Reset() { *x = Aggregation_AggregationCount{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[116] + mi := &file_deephaven_proto_table_proto_msgTypes[117] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8915,7 +9000,7 @@ func (x *Aggregation_AggregationCount) String() string { func (*Aggregation_AggregationCount) ProtoMessage() {} func (x *Aggregation_AggregationCount) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[116] + mi := &file_deephaven_proto_table_proto_msgTypes[117] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8928,7 +9013,7 @@ func (x *Aggregation_AggregationCount) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationCount.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationCount) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{36, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37, 1} } func (x *Aggregation_AggregationCount) GetColumnName() string { @@ -8949,7 +9034,7 @@ type Aggregation_AggregationRowKey struct { func (x *Aggregation_AggregationRowKey) Reset() { *x = Aggregation_AggregationRowKey{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[117] + mi := &file_deephaven_proto_table_proto_msgTypes[118] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -8962,7 +9047,7 @@ func (x *Aggregation_AggregationRowKey) String() string { func (*Aggregation_AggregationRowKey) ProtoMessage() {} func (x *Aggregation_AggregationRowKey) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[117] + mi := &file_deephaven_proto_table_proto_msgTypes[118] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -8975,7 +9060,7 @@ func (x *Aggregation_AggregationRowKey) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationRowKey.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationRowKey) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{36, 2} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37, 2} } func (x *Aggregation_AggregationRowKey) GetColumnName() string { @@ -8997,7 +9082,7 @@ type Aggregation_AggregationPartition struct { func (x *Aggregation_AggregationPartition) Reset() { *x = Aggregation_AggregationPartition{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[118] + mi := &file_deephaven_proto_table_proto_msgTypes[119] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9010,7 +9095,7 @@ func (x *Aggregation_AggregationPartition) String() string { func (*Aggregation_AggregationPartition) ProtoMessage() {} func (x *Aggregation_AggregationPartition) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[118] + mi := &file_deephaven_proto_table_proto_msgTypes[119] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9023,7 +9108,7 @@ func (x *Aggregation_AggregationPartition) ProtoReflect() protoreflect.Message { // Deprecated: Use Aggregation_AggregationPartition.ProtoReflect.Descriptor instead. func (*Aggregation_AggregationPartition) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{36, 3} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37, 3} } func (x *Aggregation_AggregationPartition) GetColumnName() string { @@ -9040,6 +9125,53 @@ func (x *Aggregation_AggregationPartition) GetIncludeGroupByColumns() bool { return false } +type Aggregation_AggregationFormula struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Selectable *Selectable `protobuf:"bytes,1,opt,name=selectable,proto3" json:"selectable,omitempty"` +} + +func (x *Aggregation_AggregationFormula) Reset() { + *x = Aggregation_AggregationFormula{} + if protoimpl.UnsafeEnabled { + mi := &file_deephaven_proto_table_proto_msgTypes[120] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Aggregation_AggregationFormula) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Aggregation_AggregationFormula) ProtoMessage() {} + +func (x *Aggregation_AggregationFormula) ProtoReflect() protoreflect.Message { + mi := &file_deephaven_proto_table_proto_msgTypes[120] + 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_AggregationFormula.ProtoReflect.Descriptor instead. +func (*Aggregation_AggregationFormula) Descriptor() ([]byte, []int) { + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{37, 4} +} + +func (x *Aggregation_AggregationFormula) GetSelectable() *Selectable { + if x != nil { + return x.Selectable + } + return nil +} + type RunChartDownsampleRequest_ZoomRange struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -9052,7 +9184,7 @@ type RunChartDownsampleRequest_ZoomRange struct { func (x *RunChartDownsampleRequest_ZoomRange) Reset() { *x = RunChartDownsampleRequest_ZoomRange{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[119] + mi := &file_deephaven_proto_table_proto_msgTypes[121] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9065,7 +9197,7 @@ func (x *RunChartDownsampleRequest_ZoomRange) String() string { func (*RunChartDownsampleRequest_ZoomRange) ProtoMessage() {} func (x *RunChartDownsampleRequest_ZoomRange) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[119] + mi := &file_deephaven_proto_table_proto_msgTypes[121] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9078,7 +9210,7 @@ func (x *RunChartDownsampleRequest_ZoomRange) ProtoReflect() protoreflect.Messag // Deprecated: Use RunChartDownsampleRequest_ZoomRange.ProtoReflect.Descriptor instead. func (*RunChartDownsampleRequest_ZoomRange) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{58, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59, 0} } func (x *RunChartDownsampleRequest_ZoomRange) GetMinDateNanos() int64 { @@ -9110,7 +9242,7 @@ type CreateInputTableRequest_InputTableKind struct { func (x *CreateInputTableRequest_InputTableKind) Reset() { *x = CreateInputTableRequest_InputTableKind{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[120] + mi := &file_deephaven_proto_table_proto_msgTypes[122] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9123,7 +9255,7 @@ func (x *CreateInputTableRequest_InputTableKind) String() string { func (*CreateInputTableRequest_InputTableKind) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[120] + mi := &file_deephaven_proto_table_proto_msgTypes[122] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9136,7 +9268,7 @@ func (x *CreateInputTableRequest_InputTableKind) ProtoReflect() protoreflect.Mes // Deprecated: Use CreateInputTableRequest_InputTableKind.ProtoReflect.Descriptor instead. func (*CreateInputTableRequest_InputTableKind) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{60, 0} } func (m *CreateInputTableRequest_InputTableKind) GetKind() isCreateInputTableRequest_InputTableKind_Kind { @@ -9202,7 +9334,7 @@ type CreateInputTableRequest_InputTableKind_InMemoryAppendOnly struct { func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) Reset() { *x = CreateInputTableRequest_InputTableKind_InMemoryAppendOnly{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[121] + mi := &file_deephaven_proto_table_proto_msgTypes[123] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9215,7 +9347,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) String() str func (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[121] + mi := &file_deephaven_proto_table_proto_msgTypes[123] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9228,7 +9360,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoReflect // Deprecated: Use CreateInputTableRequest_InputTableKind_InMemoryAppendOnly.ProtoReflect.Descriptor instead. func (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59, 0, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{60, 0, 0} } // Creates an in-memory table that supports updates and deletes by keys. @@ -9243,7 +9375,7 @@ type CreateInputTableRequest_InputTableKind_InMemoryKeyBacked struct { func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) Reset() { *x = CreateInputTableRequest_InputTableKind_InMemoryKeyBacked{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[122] + mi := &file_deephaven_proto_table_proto_msgTypes[124] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9256,7 +9388,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) String() stri func (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[122] + mi := &file_deephaven_proto_table_proto_msgTypes[124] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9269,7 +9401,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoReflect( // Deprecated: Use CreateInputTableRequest_InputTableKind_InMemoryKeyBacked.ProtoReflect.Descriptor instead. func (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59, 0, 1} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{60, 0, 1} } func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) GetKeyColumns() []string { @@ -9288,7 +9420,7 @@ type CreateInputTableRequest_InputTableKind_Blink struct { func (x *CreateInputTableRequest_InputTableKind_Blink) Reset() { *x = CreateInputTableRequest_InputTableKind_Blink{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[123] + mi := &file_deephaven_proto_table_proto_msgTypes[125] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9301,7 +9433,7 @@ func (x *CreateInputTableRequest_InputTableKind_Blink) String() string { func (*CreateInputTableRequest_InputTableKind_Blink) ProtoMessage() {} func (x *CreateInputTableRequest_InputTableKind_Blink) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[123] + mi := &file_deephaven_proto_table_proto_msgTypes[125] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9314,7 +9446,7 @@ func (x *CreateInputTableRequest_InputTableKind_Blink) ProtoReflect() protorefle // Deprecated: Use CreateInputTableRequest_InputTableKind_Blink.ProtoReflect.Descriptor instead. func (*CreateInputTableRequest_InputTableKind_Blink) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{59, 0, 2} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{60, 0, 2} } type BatchTableRequest_Operation struct { @@ -9371,7 +9503,7 @@ type BatchTableRequest_Operation struct { func (x *BatchTableRequest_Operation) Reset() { *x = BatchTableRequest_Operation{} if protoimpl.UnsafeEnabled { - mi := &file_deephaven_proto_table_proto_msgTypes[124] + mi := &file_deephaven_proto_table_proto_msgTypes[126] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -9384,7 +9516,7 @@ func (x *BatchTableRequest_Operation) String() string { func (*BatchTableRequest_Operation) ProtoMessage() {} func (x *BatchTableRequest_Operation) ProtoReflect() protoreflect.Message { - mi := &file_deephaven_proto_table_proto_msgTypes[124] + mi := &file_deephaven_proto_table_proto_msgTypes[126] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -9397,7 +9529,7 @@ func (x *BatchTableRequest_Operation) ProtoReflect() protoreflect.Message { // Deprecated: Use BatchTableRequest_Operation.ProtoReflect.Descriptor instead. func (*BatchTableRequest_Operation) Descriptor() ([]byte, []int) { - return file_deephaven_proto_table_proto_rawDescGZIP(), []int{63, 0} + return file_deephaven_proto_table_proto_rawDescGZIP(), []int{64, 0} } func (m *BatchTableRequest_Operation) GetOp() isBatchTableRequest_Operation_Op { @@ -10068,195 +10200,180 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 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, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0xa5, - 0x02, 0x0a, 0x0b, 0x4d, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x1c, - 0x0a, 0x09, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x11, 0x52, 0x09, 0x70, 0x72, 0x65, 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x60, 0x0a, 0x0d, - 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0e, 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, 0x4d, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x74, - 0x65, 0x78, 0x74, 0x2e, 0x52, 0x6f, 0x75, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4d, 0x6f, 0x64, 0x65, - 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x95, - 0x01, 0x0a, 0x0c, 0x52, 0x6f, 0x75, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4d, 0x6f, 0x64, 0x65, 0x12, - 0x1f, 0x0a, 0x1b, 0x52, 0x4f, 0x55, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x4d, 0x4f, 0x44, 0x45, - 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, - 0x12, 0x06, 0x0a, 0x02, 0x55, 0x50, 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x44, 0x4f, 0x57, 0x4e, - 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x45, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x12, - 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f, 0x4f, 0x52, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, 0x48, 0x41, - 0x4c, 0x46, 0x5f, 0x55, 0x50, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x41, 0x4c, 0x46, 0x5f, - 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x06, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x41, 0x4c, 0x46, 0x5f, 0x45, - 0x56, 0x45, 0x4e, 0x10, 0x07, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x4e, 0x45, 0x43, 0x45, 0x53, - 0x53, 0x41, 0x52, 0x59, 0x10, 0x08, 0x22, 0x8e, 0x03, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x62, - 0x0a, 0x05, 0x74, 0x69, 0x63, 0x6b, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 0x2e, + 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0x28, + 0x0a, 0x0a, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x12, 0x0a, 0x03, + 0x72, 0x61, 0x77, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x77, + 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xa5, 0x02, 0x0a, 0x0b, 0x4d, 0x61, 0x74, + 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x72, 0x65, 0x63, + 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x11, 0x52, 0x09, 0x70, 0x72, 0x65, + 0x63, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x60, 0x0a, 0x0d, 0x72, 0x6f, 0x75, 0x6e, 0x64, 0x69, + 0x6e, 0x67, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, - 0x53, 0x63, 0x61, 0x6c, 0x65, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, - 0x6e, 0x64, 0x6f, 0x77, 0x54, 0x69, 0x63, 0x6b, 0x73, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, 0x63, - 0x6b, 0x73, 0x12, 0x5f, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, - 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52, 0x04, 0x74, - 0x69, 0x6d, 0x65, 0x1a, 0x2b, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, - 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x54, 0x69, 0x63, 0x6b, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x69, - 0x63, 0x6b, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x05, 0x74, 0x69, 0x63, 0x6b, 0x73, - 0x1a, 0x7d, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, - 0x6f, 0x77, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1a, - 0x0a, 0x05, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, - 0x01, 0x48, 0x00, 0x52, 0x05, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x64, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0e, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, - 0x74, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x08, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x42, - 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xbd, 0x04, 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x56, 0x0a, - 0x0d, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, - 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, - 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0b, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x54, 0x0a, 0x0c, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6e, 0x5f, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x69, 0x6f, + 0x63, 0x2e, 0x4d, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x2e, 0x52, 0x6f, + 0x75, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x0c, 0x72, 0x6f, 0x75, 0x6e, + 0x64, 0x69, 0x6e, 0x67, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x95, 0x01, 0x0a, 0x0c, 0x52, 0x6f, 0x75, + 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x52, 0x4f, 0x55, + 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x5f, 0x4d, 0x4f, 0x44, 0x45, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, + 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x55, 0x50, + 0x10, 0x01, 0x12, 0x08, 0x0a, 0x04, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, + 0x43, 0x45, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f, + 0x4f, 0x52, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, 0x48, 0x41, 0x4c, 0x46, 0x5f, 0x55, 0x50, 0x10, + 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x41, 0x4c, 0x46, 0x5f, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x06, + 0x12, 0x0d, 0x0a, 0x09, 0x48, 0x41, 0x4c, 0x46, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x10, 0x07, 0x12, + 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x4e, 0x45, 0x43, 0x45, 0x53, 0x53, 0x41, 0x52, 0x59, 0x10, 0x08, + 0x22, 0x8e, 0x03, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x62, 0x0a, 0x05, 0x74, 0x69, 0x63, 0x6b, + 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 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, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x54, 0x69, + 0x63, 0x6b, 0x73, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, 0x63, 0x6b, 0x73, 0x12, 0x5f, 0x0a, 0x04, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 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, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, + 0x6c, 0x65, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x1a, 0x2b, 0x0a, + 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x54, + 0x69, 0x63, 0x6b, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x69, 0x63, 0x6b, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x01, 0x52, 0x05, 0x74, 0x69, 0x63, 0x6b, 0x73, 0x1a, 0x7d, 0x0a, 0x12, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x54, 0x69, 0x6d, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x1a, 0x0a, 0x05, 0x6e, 0x61, 0x6e, 0x6f, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x48, 0x00, 0x52, 0x05, 0x6e, + 0x61, 0x6e, 0x6f, 0x73, 0x12, 0x29, 0x0a, 0x0f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, + 0x0e, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x42, + 0x08, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x22, 0xbd, 0x04, 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, + 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x56, 0x0a, 0x0d, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, + 0x6c, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, + 0x6f, 0x72, 0x52, 0x0b, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, + 0x54, 0x0a, 0x0c, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6e, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, + 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0a, 0x6f, 0x6e, 0x4e, 0x61, 0x6e, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x54, 0x0a, 0x0c, 0x6f, 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, - 0x0a, 0x6f, 0x6e, 0x4e, 0x61, 0x6e, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x54, 0x0a, 0x0c, 0x6f, - 0x6e, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x74, 0x69, 0x6d, 0x65, 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, - 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0a, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, 0x54, 0x69, 0x6d, - 0x65, 0x12, 0x67, 0x0a, 0x16, 0x6f, 0x6e, 0x5f, 0x6e, 0x65, 0x67, 0x61, 0x74, 0x69, 0x76, 0x65, - 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, - 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x13, 0x6f, 0x6e, 0x4e, 0x65, 0x67, 0x61, 0x74, 0x69, 0x76, - 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x5f, 0x0a, 0x12, 0x6f, 0x6e, - 0x5f, 0x7a, 0x65, 0x72, 0x6f, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x18, 0x05, 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, 0x42, 0x61, 0x64, 0x44, 0x61, - 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0f, 0x6f, 0x6e, 0x5a, 0x65, - 0x72, 0x6f, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x5a, 0x0a, 0x11, 0x62, - 0x69, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, - 0x18, 0x06, 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, 0x4d, 0x61, 0x74, 0x68, 0x43, - 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0f, 0x62, 0x69, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x22, 0x74, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x5c, 0x0a, 0x0d, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 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, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x4e, 0x75, 0x6c, 0x6c, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, - 0x0c, 0x6e, 0x75, 0x6c, 0x6c, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0xeb, 0x3e, - 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 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, 0x5c, 0x0a, 0x07, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, - 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x64, 0x0a, 0x0a, 0x6f, 0x70, 0x65, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x69, 0x6f, + 0x0a, 0x6f, 0x6e, 0x4e, 0x75, 0x6c, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x67, 0x0a, 0x16, 0x6f, + 0x6e, 0x5f, 0x6e, 0x65, 0x67, 0x61, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 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, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x0a, 0x6f, 0x70, 0x65, 0x72, 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, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0xc9, 0x04, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x0f, 0x75, - 0x73, 0x65, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x0e, 0x75, 0x73, 0x65, 0x52, 0x65, 0x64, 0x69, 0x72, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x12, 0x2a, 0x0a, 0x0e, 0x63, 0x68, 0x75, - 0x6e, 0x6b, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x05, 0x48, 0x01, 0x52, 0x0d, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, - 0x74, 0x79, 0x88, 0x01, 0x01, 0x12, 0x4d, 0x0a, 0x21, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x74, 0x61, - 0x74, 0x69, 0x63, 0x5f, 0x73, 0x70, 0x61, 0x72, 0x73, 0x65, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, - 0x79, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x68, 0x65, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, - 0x48, 0x02, 0x52, 0x1d, 0x6d, 0x61, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x63, 0x53, 0x70, 0x61, - 0x72, 0x73, 0x65, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4f, 0x76, 0x65, 0x72, 0x68, 0x65, 0x61, - 0x64, 0x88, 0x01, 0x01, 0x12, 0x3a, 0x0a, 0x17, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, - 0x68, 0x61, 0x73, 0x68, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x03, 0x52, 0x14, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, - 0x48, 0x61, 0x73, 0x68, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x88, 0x01, 0x01, - 0x12, 0x33, 0x0a, 0x13, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x6c, 0x6f, 0x61, 0x64, - 0x5f, 0x66, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x48, 0x04, 0x52, - 0x11, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x63, 0x74, - 0x6f, 0x72, 0x88, 0x01, 0x01, 0x12, 0x31, 0x0a, 0x12, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, - 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x01, 0x48, 0x05, 0x52, 0x10, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x46, - 0x61, 0x63, 0x74, 0x6f, 0x72, 0x88, 0x01, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x6d, 0x61, 0x74, 0x68, - 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x07, 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, 0x4d, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, - 0x6d, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x42, 0x12, 0x0a, 0x10, 0x5f, - 0x75, 0x73, 0x65, 0x5f, 0x72, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x42, - 0x11, 0x0a, 0x0f, 0x5f, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, - 0x74, 0x79, 0x42, 0x24, 0x0a, 0x22, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, - 0x63, 0x5f, 0x73, 0x70, 0x61, 0x72, 0x73, 0x65, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, - 0x6f, 0x76, 0x65, 0x72, 0x68, 0x65, 0x61, 0x64, 0x42, 0x1a, 0x0a, 0x18, 0x5f, 0x69, 0x6e, 0x69, - 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, - 0x73, 0x69, 0x7a, 0x65, 0x42, 0x16, 0x0a, 0x14, 0x5f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, - 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x42, 0x15, 0x0a, 0x13, - 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x61, 0x63, - 0x74, 0x6f, 0x72, 0x1a, 0x85, 0x37, 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x6d, 0x0a, 0x06, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x53, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x48, 0x00, - 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0xf8, 0x35, 0x0a, 0x0e, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x74, 0x0a, 0x04, 0x73, - 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x60, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 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, 0xce, 0x34, 0x0a, 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, - 0x70, 0x65, 0x63, 0x12, 0x8a, 0x01, 0x0a, 0x03, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x76, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, - 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, - 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x53, 0x75, 0x6d, 0x48, 0x00, 0x52, 0x03, 0x73, 0x75, 0x6d, - 0x12, 0x8a, 0x01, 0x0a, 0x03, 0x6d, 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x76, + 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, + 0x13, 0x6f, 0x6e, 0x4e, 0x65, 0x67, 0x61, 0x74, 0x69, 0x76, 0x65, 0x44, 0x65, 0x6c, 0x74, 0x61, + 0x54, 0x69, 0x6d, 0x65, 0x12, 0x5f, 0x0a, 0x12, 0x6f, 0x6e, 0x5f, 0x7a, 0x65, 0x72, 0x6f, 0x5f, + 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 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, 0x42, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, + 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0f, 0x6f, 0x6e, 0x5a, 0x65, 0x72, 0x6f, 0x44, 0x65, 0x6c, 0x74, + 0x61, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x5a, 0x0a, 0x11, 0x62, 0x69, 0x67, 0x5f, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x18, 0x06, 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, 0x4d, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, + 0x52, 0x0f, 0x62, 0x69, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, + 0x74, 0x22, 0x74, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x44, 0x65, 0x6c, + 0x74, 0x61, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x5c, 0x0a, 0x0d, 0x6e, 0x75, 0x6c, + 0x6c, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, + 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4e, 0x75, 0x6c, + 0x6c, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x0c, 0x6e, 0x75, 0x6c, 0x6c, 0x42, + 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0xeb, 0x3e, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, - 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x6d, 0x69, 0x6e, 0x12, 0x8a, 0x01, - 0x0a, 0x03, 0x6d, 0x61, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x76, 0x2e, 0x69, 0x6f, + 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, 0x5c, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x64, 0x0a, 0x0a, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x04, 0x20, 0x03, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x6f, 0x70, 0x65, + 0x72, 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, 0x05, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x1a, 0xc9, 0x04, 0x0a, 0x0f, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x0a, 0x0f, 0x75, 0x73, 0x65, 0x5f, 0x72, 0x65, 0x64, + 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, + 0x52, 0x0e, 0x75, 0x73, 0x65, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x88, 0x01, 0x01, 0x12, 0x2a, 0x0a, 0x0e, 0x63, 0x68, 0x75, 0x6e, 0x6b, 0x5f, 0x63, 0x61, 0x70, + 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x48, 0x01, 0x52, 0x0d, 0x63, + 0x68, 0x75, 0x6e, 0x6b, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x88, 0x01, 0x01, 0x12, + 0x4d, 0x0a, 0x21, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, 0x63, 0x5f, 0x73, 0x70, + 0x61, 0x72, 0x73, 0x65, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6f, 0x76, 0x65, 0x72, + 0x68, 0x65, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x01, 0x48, 0x02, 0x52, 0x1d, 0x6d, 0x61, + 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x63, 0x53, 0x70, 0x61, 0x72, 0x73, 0x65, 0x4d, 0x65, 0x6d, + 0x6f, 0x72, 0x79, 0x4f, 0x76, 0x65, 0x72, 0x68, 0x65, 0x61, 0x64, 0x88, 0x01, 0x01, 0x12, 0x3a, + 0x0a, 0x17, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x5f, 0x74, + 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, + 0x03, 0x52, 0x14, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x48, 0x61, 0x73, 0x68, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x88, 0x01, 0x01, 0x12, 0x33, 0x0a, 0x13, 0x6d, 0x61, + 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x61, 0x63, 0x74, 0x6f, + 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x01, 0x48, 0x04, 0x52, 0x11, 0x6d, 0x61, 0x78, 0x69, 0x6d, + 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x88, 0x01, 0x01, 0x12, + 0x31, 0x0a, 0x12, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, + 0x61, 0x63, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x48, 0x05, 0x52, 0x10, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x46, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x88, + 0x01, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x6d, 0x61, 0x74, 0x68, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x65, + 0x78, 0x74, 0x18, 0x07, 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, 0x4d, 0x61, 0x74, + 0x68, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x52, 0x0b, 0x6d, 0x61, 0x74, 0x68, 0x43, 0x6f, + 0x6e, 0x74, 0x65, 0x78, 0x74, 0x42, 0x12, 0x0a, 0x10, 0x5f, 0x75, 0x73, 0x65, 0x5f, 0x72, 0x65, + 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x63, 0x68, + 0x75, 0x6e, 0x6b, 0x5f, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x42, 0x24, 0x0a, 0x22, + 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, 0x63, 0x5f, 0x73, 0x70, 0x61, 0x72, + 0x73, 0x65, 0x5f, 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x68, 0x65, + 0x61, 0x64, 0x42, 0x1a, 0x0a, 0x18, 0x5f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x68, + 0x61, 0x73, 0x68, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x42, 0x16, + 0x0a, 0x14, 0x5f, 0x6d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x5f, + 0x66, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x1a, 0x85, 0x37, + 0x0a, 0x11, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x6d, 0x0a, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x53, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x48, 0x00, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x1a, 0xf8, 0x35, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x74, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x60, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 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, 0xce, 0x34, 0x0a, + 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x12, 0x8a, 0x01, + 0x0a, 0x03, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x76, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, @@ -10264,218 +10381,212 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, - 0x4d, 0x61, 0x78, 0x48, 0x00, 0x52, 0x03, 0x6d, 0x61, 0x78, 0x12, 0x96, 0x01, 0x0a, 0x07, 0x70, - 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x7a, 0x2e, 0x69, + 0x53, 0x75, 0x6d, 0x48, 0x00, 0x52, 0x03, 0x73, 0x75, 0x6d, 0x12, 0x8a, 0x01, 0x0a, 0x03, 0x6d, + 0x69, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x76, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6e, + 0x48, 0x00, 0x52, 0x03, 0x6d, 0x69, 0x6e, 0x12, 0x8a, 0x01, 0x0a, 0x03, 0x6d, 0x61, 0x78, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x76, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, + 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x4d, 0x61, 0x78, 0x48, 0x00, 0x52, + 0x03, 0x6d, 0x61, 0x78, 0x12, 0x96, 0x01, 0x0a, 0x07, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x7a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x74, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x12, 0x83, 0x01, + 0x0a, 0x04, 0x66, 0x69, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, - 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x48, 0x00, 0x52, 0x07, 0x70, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x74, 0x12, 0x83, 0x01, 0x0a, 0x04, 0x66, 0x69, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x6d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, - 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x46, 0x69, 0x6c, - 0x6c, 0x48, 0x00, 0x52, 0x04, 0x66, 0x69, 0x6c, 0x6c, 0x12, 0x80, 0x01, 0x0a, 0x03, 0x65, 0x6d, - 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, - 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x45, 0x6d, 0x61, 0x48, 0x00, 0x52, 0x03, 0x65, 0x6d, 0x61, 0x12, 0x96, 0x01, 0x0a, - 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x75, 0x6d, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, - 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, - 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x75, 0x6d, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, 0x69, - 0x6e, 0x67, 0x53, 0x75, 0x6d, 0x12, 0x9c, 0x01, 0x0a, 0x0d, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, - 0x67, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x75, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x12, 0x96, 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, - 0x5f, 0x61, 0x76, 0x67, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, - 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x67, 0x48, - 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x67, 0x12, 0x96, 0x01, - 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x69, 0x6e, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, - 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, - 0x69, 0x6e, 0x67, 0x4d, 0x69, 0x6e, 0x12, 0x96, 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, - 0x6e, 0x67, 0x5f, 0x6d, 0x61, 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x46, 0x69, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x04, 0x66, + 0x69, 0x6c, 0x6c, 0x12, 0x80, 0x01, 0x0a, 0x03, 0x65, 0x6d, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x6c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, + 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x61, 0x48, + 0x00, 0x52, 0x03, 0x65, 0x6d, 0x61, 0x12, 0x96, 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, + 0x6e, 0x67, 0x5f, 0x73, 0x75, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x61, - 0x78, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x61, 0x78, 0x12, - 0xa2, 0x01, 0x0a, 0x0f, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x64, - 0x75, 0x63, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x77, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x74, 0x12, 0x86, 0x01, 0x0a, 0x05, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x0d, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x75, + 0x6d, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x75, 0x6d, 0x12, + 0x9c, 0x01, 0x0a, 0x0d, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x75, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x48, 0x00, + 0x52, 0x0c, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x96, + 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x61, 0x76, 0x67, 0x18, 0x09, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x44, - 0x65, 0x6c, 0x74, 0x61, 0x48, 0x00, 0x52, 0x05, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x80, 0x01, - 0x0a, 0x03, 0x65, 0x6d, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6c, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x73, 0x48, 0x00, 0x52, 0x03, 0x65, 0x6d, 0x73, - 0x12, 0x87, 0x01, 0x0a, 0x06, 0x65, 0x6d, 0x5f, 0x6d, 0x69, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x6e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, - 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4d, 0x69, - 0x6e, 0x48, 0x00, 0x52, 0x05, 0x65, 0x6d, 0x4d, 0x69, 0x6e, 0x12, 0x87, 0x01, 0x0a, 0x06, 0x65, - 0x6d, 0x5f, 0x6d, 0x61, 0x78, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6e, 0x2e, 0x69, 0x6f, - 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4d, 0x61, 0x78, 0x48, 0x00, 0x52, 0x05, 0x65, - 0x6d, 0x4d, 0x61, 0x78, 0x12, 0x87, 0x01, 0x0a, 0x06, 0x65, 0x6d, 0x5f, 0x73, 0x74, 0x64, 0x18, - 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x45, 0x6d, 0x53, 0x74, 0x64, 0x48, 0x00, 0x52, 0x05, 0x65, 0x6d, 0x53, 0x74, 0x64, 0x12, 0x9c, - 0x01, 0x0a, 0x0d, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, - 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x75, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, + 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x67, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, + 0x6c, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x67, 0x12, 0x96, 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, + 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x69, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, + 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, + 0x69, 0x6e, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x69, 0x6e, + 0x12, 0x96, 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x6d, 0x61, 0x78, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x48, 0x00, 0x52, - 0x0c, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x96, 0x01, - 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x64, 0x18, 0x13, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x61, 0x78, 0x48, 0x00, 0x52, 0x0a, 0x72, + 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x61, 0x78, 0x12, 0xa2, 0x01, 0x0a, 0x0f, 0x72, 0x6f, + 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x18, 0x0c, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x77, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, - 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x64, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, - 0x69, 0x6e, 0x67, 0x53, 0x74, 0x64, 0x12, 0x99, 0x01, 0x0a, 0x0c, 0x72, 0x6f, 0x6c, 0x6c, 0x69, - 0x6e, 0x67, 0x5f, 0x77, 0x61, 0x76, 0x67, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x74, 0x2e, - 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x57, - 0x41, 0x76, 0x67, 0x48, 0x00, 0x52, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x57, 0x61, - 0x76, 0x67, 0x12, 0xa2, 0x01, 0x0a, 0x0f, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x66, - 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x77, 0x2e, 0x69, - 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x46, 0x6f, - 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x0e, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, - 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x1a, 0x17, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x53, 0x75, 0x6d, - 0x1a, 0x17, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, - 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6e, 0x1a, 0x17, 0x0a, 0x15, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x4d, - 0x61, 0x78, 0x1a, 0x1b, 0x0a, 0x19, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, - 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x1a, - 0x0e, 0x0a, 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x46, 0x69, 0x6c, 0x6c, 0x1a, - 0xb8, 0x01, 0x0a, 0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x61, 0x12, - 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x59, 0x0a, 0x0c, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, + 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x48, 0x00, 0x52, 0x0e, + 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x12, 0x86, + 0x01, 0x0a, 0x05, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6e, + 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x48, 0x00, + 0x52, 0x05, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x80, 0x01, 0x0a, 0x03, 0x65, 0x6d, 0x73, 0x18, + 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, - 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xb8, 0x01, 0x0a, 0x0b, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, + 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x45, 0x6d, 0x73, 0x48, 0x00, 0x52, 0x03, 0x65, 0x6d, 0x73, 0x12, 0x87, 0x01, 0x0a, 0x06, 0x65, + 0x6d, 0x5f, 0x6d, 0x69, 0x6e, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x45, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x59, 0x0a, 0x0c, 0x77, 0x69, - 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4d, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x05, 0x65, + 0x6d, 0x4d, 0x69, 0x6e, 0x12, 0x87, 0x01, 0x0a, 0x06, 0x65, 0x6d, 0x5f, 0x6d, 0x61, 0x78, 0x18, + 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x6e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, + 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x45, 0x6d, 0x4d, 0x61, 0x78, 0x48, 0x00, 0x52, 0x05, 0x65, 0x6d, 0x4d, 0x61, 0x78, 0x12, 0x87, + 0x01, 0x0a, 0x06, 0x65, 0x6d, 0x5f, 0x73, 0x74, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x6e, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, + 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x53, 0x74, 0x64, 0x48, + 0x00, 0x52, 0x05, 0x65, 0x6d, 0x53, 0x74, 0x64, 0x12, 0x9c, 0x01, 0x0a, 0x0d, 0x72, 0x6f, 0x6c, + 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x75, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x57, 0x69, 0x6e, - 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, - 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xba, 0x01, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x45, 0x6d, 0x4d, 0x69, 0x6e, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 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, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, - 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x59, 0x0a, 0x0c, 0x77, 0x69, 0x6e, 0x64, 0x6f, - 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, + 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, + 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x72, 0x6f, 0x6c, 0x6c, 0x69, + 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x96, 0x01, 0x0a, 0x0b, 0x72, 0x6f, 0x6c, 0x6c, + 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x64, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x73, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, - 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, - 0x6c, 0x65, 0x1a, 0xba, 0x01, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, - 0x6d, 0x4d, 0x61, 0x78, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, + 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, + 0x74, 0x64, 0x48, 0x00, 0x52, 0x0a, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x64, + 0x12, 0x99, 0x01, 0x0a, 0x0c, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x77, 0x61, 0x76, + 0x67, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x74, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x57, 0x41, 0x76, 0x67, 0x48, 0x00, 0x52, + 0x0b, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x57, 0x61, 0x76, 0x67, 0x12, 0xa2, 0x01, 0x0a, + 0x0f, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, + 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x77, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, + 0x68, 0x61, 0x76, 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, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x48, + 0x00, 0x52, 0x0e, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, + 0x61, 0x1a, 0x17, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, + 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x53, 0x75, 0x6d, 0x1a, 0x17, 0x0a, 0x15, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, + 0x4d, 0x69, 0x6e, 0x1a, 0x17, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, + 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x4d, 0x61, 0x78, 0x1a, 0x1b, 0x0a, 0x19, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, + 0x76, 0x65, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x1a, 0x0e, 0x0a, 0x0c, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x46, 0x69, 0x6c, 0x6c, 0x1a, 0xb8, 0x01, 0x0a, 0x0b, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x61, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 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, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x59, 0x0a, 0x0c, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, + 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xb8, 0x01, 0x0a, 0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x45, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, @@ -10486,8 +10597,8 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, - 0xba, 0x01, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x53, 0x74, - 0x64, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, + 0xba, 0x01, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4d, 0x69, + 0x6e, 0x12, 0x4e, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, @@ -10497,30 +10608,83 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 0x70, 0x68, 0x61, 0x76, 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, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, - 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0x62, 0x0a, 0x0d, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x51, 0x0a, - 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x44, 0x65, 0x6c, 0x74, 0x61, - 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x1a, 0xe8, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, - 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x75, 0x6d, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, - 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, - 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, - 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, - 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, - 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xba, 0x01, 0x0a, + 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4d, 0x61, 0x78, 0x12, 0x4e, + 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x59, + 0x0a, 0x0c, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, + 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, + 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xba, 0x01, 0x0a, 0x0d, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x53, 0x74, 0x64, 0x12, 0x4e, 0x0a, 0x07, 0x6f, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x45, 0x6d, 0x4f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x59, 0x0a, 0x0c, 0x77, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x0b, 0x77, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0x62, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x51, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x01, 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, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0xe8, 0x01, 0x0a, 0x12, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x75, + 0x6d, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, - 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, - 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xea, 0x01, 0x0a, 0x14, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, + 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, + 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, + 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, + 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, + 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xea, 0x01, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x68, + 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, + 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, + 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, + 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, + 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, + 0x6c, 0x65, 0x1a, 0xe8, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, + 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x67, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, + 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, + 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, + 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, + 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xe8, 0x01, + 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, + 0x67, 0x4d, 0x69, 0x6e, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, @@ -10534,7 +10698,7 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xe8, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x67, 0x12, + 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x61, 0x78, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, @@ -10548,131 +10712,128 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, - 0x61, 0x6c, 0x65, 0x1a, 0xe8, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x4d, 0x69, 0x6e, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, - 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, - 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, - 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, - 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, - 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, - 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, - 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xe8, - 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, - 0x6e, 0x67, 0x4d, 0x61, 0x78, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, - 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, - 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, - 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, - 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, - 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, - 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, - 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xec, 0x01, 0x0a, 0x16, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, - 0x64, 0x75, 0x63, 0x74, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, - 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, - 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, - 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, - 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, - 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, + 0x61, 0x6c, 0x65, 0x1a, 0xec, 0x01, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x74, 0x12, 0x68, + 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, - 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, - 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xea, 0x01, 0x0a, 0x14, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, - 0x74, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, - 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, - 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, - 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, - 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, - 0x61, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, 0x64, + 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, + 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, + 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, + 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, + 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, + 0x6c, 0x65, 0x1a, 0xea, 0x01, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, + 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x68, 0x0a, 0x14, 0x72, + 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, + 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, - 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, - 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0xe8, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x64, 0x12, 0x68, 0x0a, 0x14, - 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, - 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, - 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, - 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, + 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, + 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, + 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, + 0xe8, 0x01, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, + 0x69, 0x6e, 0x67, 0x53, 0x74, 0x64, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, + 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, - 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, - 0x1a, 0x8e, 0x02, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, - 0x6c, 0x69, 0x6e, 0x67, 0x57, 0x41, 0x76, 0x67, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, - 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, - 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, - 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, - 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, - 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, + 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, + 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x1a, 0x8e, 0x02, 0x0a, 0x13, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x57, 0x41, + 0x76, 0x67, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, + 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, - 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, - 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x1a, 0xa7, 0x02, 0x0a, 0x16, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, - 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x68, 0x0a, 0x14, - 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, - 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x69, 0x6f, 0x2e, + 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, + 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, + 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, + 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, - 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, - 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, + 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, + 0x65, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0xa7, 0x02, 0x0a, 0x16, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x46, + 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x68, 0x0a, 0x14, 0x72, 0x65, 0x76, 0x65, 0x72, 0x73, + 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x01, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, - 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, - 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, - 0x12, 0x18, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x61, - 0x72, 0x61, 0x6d, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x06, 0x0a, 0x04, 0x74, - 0x79, 0x70, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x15, - 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 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, 0x21, 0x0a, - 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, - 0x22, 0xcf, 0x01, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x72, 0x65, + 0x76, 0x65, 0x72, 0x73, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, + 0x12, 0x68, 0x0a, 0x14, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, + 0x6f, 0x77, 0x5f, 0x73, 0x63, 0x61, 0x6c, 0x65, 0x18, 0x02, 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x57, 0x69, 0x6e, 0x64, 0x6f, + 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x52, 0x12, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x57, + 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x53, 0x63, 0x61, 0x6c, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x6f, + 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x66, 0x6f, 0x72, + 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x5f, 0x74, 0x6f, + 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x42, 0x06, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x15, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, + 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 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, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0xcf, 0x01, 0x0a, 0x12, 0x44, + 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 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, 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x22, 0xd2, 0x01, 0x0a, + 0x1e, 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, 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, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, + 0x73, 0x22, 0xca, 0x01, 0x0a, 0x11, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 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, @@ -10683,281 +10844,208 @@ var file_deephaven_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, 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, - 0x21, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, - 0x65, 0x73, 0x22, 0xd2, 0x01, 0x0a, 0x1e, 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, 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, 0x18, 0x0a, - 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, - 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22, 0xca, 0x01, 0x0a, 0x11, 0x48, 0x65, 0x61, 0x64, - 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, + 0x1d, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x22, 0xff, + 0x01, 0x0a, 0x13, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 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, 0x1d, + 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x12, + 0x42, 0x02, 0x30, 0x01, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x31, 0x0a, + 0x15, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x12, 0x67, 0x72, + 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x73, + 0x22, 0xf3, 0x01, 0x0a, 0x0e, 0x55, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 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, 0x1b, 0x0a, 0x09, 0x6e, + 0x75, 0x6c, 0x6c, 0x5f, 0x66, 0x69, 0x6c, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, + 0x6e, 0x75, 0x6c, 0x6c, 0x46, 0x69, 0x6c, 0x6c, 0x12, 0x2c, 0x0a, 0x12, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x75, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x55, + 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xcd, 0x01, 0x0a, 0x12, 0x4d, 0x65, 0x72, 0x67, 0x65, + 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 0x1d, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, - 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x07, 0x6e, 0x75, 0x6d, - 0x52, 0x6f, 0x77, 0x73, 0x22, 0xff, 0x01, 0x0a, 0x13, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, - 0x61, 0x69, 0x6c, 0x42, 0x79, 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, + 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x50, 0x0a, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x69, 0x64, 0x73, 0x18, 0x02, 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, 0x54, 0x61, + 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x09, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6b, 0x65, 0x79, 0x5f, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6b, 0x65, 0x79, + 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x22, 0xae, 0x01, 0x0a, 0x14, 0x53, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 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, 0x22, 0xfb, 0x02, 0x0a, 0x18, 0x53, 0x6e, 0x61, 0x70, + 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 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, 0x61, 0x62, 0x6c, - 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x08, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x6f, 0x77, 0x73, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x07, 0x6e, 0x75, 0x6d, 0x52, - 0x6f, 0x77, 0x73, 0x12, 0x31, 0x0a, 0x15, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x12, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x53, 0x70, 0x65, 0x63, 0x73, 0x22, 0xf3, 0x01, 0x0a, 0x0e, 0x55, 0x6e, 0x67, 0x72, 0x6f, - 0x75, 0x70, 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, 0x1b, 0x0a, 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x66, 0x69, 0x6c, 0x6c, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x46, 0x69, 0x6c, 0x6c, 0x12, 0x2c, - 0x0a, 0x12, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x75, 0x6e, 0x67, - 0x72, 0x6f, 0x75, 0x70, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x55, 0x6e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xcd, 0x01, 0x0a, - 0x12, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 0x50, 0x0a, 0x0a, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 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, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, - 0x63, 0x65, 0x52, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x73, 0x12, 0x1d, 0x0a, - 0x0a, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x6b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x22, 0xae, 0x01, 0x0a, - 0x14, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 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, 0x22, 0xfb, 0x02, - 0x0a, 0x18, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 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, + 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, + 0x62, 0x61, 0x73, 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, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x49, 0x64, 0x12, 0x4a, 0x0a, 0x07, 0x62, 0x61, 0x73, 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, 0x06, 0x62, 0x61, 0x73, 0x65, 0x49, 0x64, 0x12, 0x50, - 0x0a, 0x0a, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 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, 0x09, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x49, 0x64, - 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x07, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x69, 0x6e, - 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, - 0x0b, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x12, 0x18, 0x0a, 0x07, - 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x68, - 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, - 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xed, 0x02, 0x0a, 0x16, - 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 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, + 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, + 0x52, 0x06, 0x62, 0x61, 0x73, 0x65, 0x49, 0x64, 0x12, 0x50, 0x0a, 0x0a, 0x74, 0x72, 0x69, 0x67, + 0x67, 0x65, 0x72, 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, - 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x65, - 0x72, 0x76, 0x65, 0x5f, 0x62, 0x69, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, - 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x42, 0x69, 0x74, 0x73, 0x22, 0xcc, 0x02, 0x0a, 0x18, - 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, - 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, + 0x09, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x69, 0x6e, + 0x69, 0x74, 0x69, 0x61, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x6e, 0x69, + 0x74, 0x69, 0x61, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x69, 0x6e, 0x63, 0x72, 0x65, 0x6d, 0x65, 0x6e, + 0x74, 0x61, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x63, 0x72, 0x65, + 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, + 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, + 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xed, 0x02, 0x0a, 0x16, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, + 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, + 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, + 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, + 0x64, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x5f, 0x62, 0x69, + 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, + 0x65, 0x42, 0x69, 0x74, 0x73, 0x22, 0xcc, 0x02, 0x0a, 0x18, 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, + 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 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, 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, - 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, - 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x22, 0xca, 0x02, 0x0a, 0x16, 0x45, - 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, + 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, 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, + 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, + 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, + 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, + 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, + 0x6f, 0x41, 0x64, 0x64, 0x22, 0xca, 0x02, 0x0a, 0x16, 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, + 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, + 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, + 0x64, 0x22, 0xc9, 0x02, 0x0a, 0x15, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, + 0x62, 0x6c, 0x65, 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, 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, - 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, - 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x22, 0xc9, 0x02, 0x0a, 0x15, 0x4c, 0x65, 0x66, 0x74, - 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, + 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, 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, + 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, 0x28, 0x0a, + 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, + 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x22, 0x99, 0x04, + 0x0a, 0x15, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, - 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, - 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, - 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, - 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, - 0x41, 0x64, 0x64, 0x22, 0x99, 0x04, 0x0a, 0x15, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, - 0x54, 0x61, 0x62, 0x6c, 0x65, 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, + 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, 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, + 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, + 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x12, 0x6b, 0x0a, 0x10, 0x61, 0x73, 0x5f, + 0x6f, 0x66, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x42, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, - 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, - 0x74, 0x63, 0x68, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, - 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x05, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x12, - 0x6b, 0x0a, 0x10, 0x61, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x72, - 0x75, 0x6c, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x42, 0x2e, 0x69, 0x6f, 0x2e, 0x64, - 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0d, 0x61, - 0x73, 0x4f, 0x66, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x75, 0x6c, 0x65, 0x22, 0x5d, 0x0a, 0x09, - 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x45, 0x53, - 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x0d, - 0x0a, 0x09, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x01, 0x12, 0x16, 0x0a, - 0x12, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x45, 0x51, - 0x55, 0x41, 0x4c, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, - 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x03, 0x1a, 0x02, 0x18, 0x01, 0x3a, 0x02, 0x18, 0x01, 0x22, - 0xee, 0x02, 0x0a, 0x12, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 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, 0x2e, 0x0a, 0x13, 0x65, 0x78, 0x61, 0x63, - 0x74, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x65, 0x78, 0x61, 0x63, 0x74, 0x4d, 0x61, 0x74, 0x63, - 0x68, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x20, 0x0a, 0x0c, 0x61, 0x73, 0x5f, 0x6f, - 0x66, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, - 0x61, 0x73, 0x4f, 0x66, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, - 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, - 0x22, 0xb0, 0x01, 0x0a, 0x0e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x49, 0x6e, - 0x70, 0x75, 0x74, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, - 0x18, 0x01, 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, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, - 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, - 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, - 0x41, 0x64, 0x64, 0x22, 0xbf, 0x01, 0x0a, 0x16, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, - 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, 0x5d, 0x0a, 0x11, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, - 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 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, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x49, - 0x6e, 0x70, 0x75, 0x74, 0x52, 0x0f, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x49, - 0x6e, 0x70, 0x75, 0x74, 0x73, 0x22, 0xf8, 0x07, 0x0a, 0x16, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, - 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 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, 0x2e, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0d, 0x61, 0x73, 0x4f, 0x66, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x52, 0x75, 0x6c, 0x65, 0x22, 0x5d, 0x0a, 0x09, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, + 0x75, 0x6c, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, + 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x45, 0x53, 0x53, + 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x47, 0x52, 0x45, 0x41, 0x54, + 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, 0x02, 0x12, + 0x10, 0x0a, 0x0c, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, + 0x03, 0x1a, 0x02, 0x18, 0x01, 0x3a, 0x02, 0x18, 0x01, 0x22, 0xee, 0x02, 0x0a, 0x12, 0x41, 0x6a, + 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 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, @@ -10975,1532 +11063,1591 @@ var file_deephaven_proto_table_proto_rawDesc = []byte{ 0x49, 0x64, 0x12, 0x2e, 0x0a, 0x13, 0x65, 0x78, 0x61, 0x63, 0x74, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x65, 0x78, 0x61, 0x63, 0x74, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x73, 0x12, 0x2a, 0x0a, 0x11, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, - 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6c, - 0x65, 0x66, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x72, - 0x0a, 0x10, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x72, 0x75, - 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x48, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, - 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, - 0x6c, 0x65, 0x52, 0x0e, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, - 0x6c, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, - 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, - 0x72, 0x69, 0x67, 0x68, 0x74, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x12, 0x6c, 0x0a, 0x0e, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x75, - 0x6c, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 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, 0x61, 0x6e, - 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x6e, 0x64, 0x52, 0x75, 0x6c, 0x65, - 0x52, 0x0c, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x6e, 0x64, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x26, - 0x0a, 0x0f, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6c, 0x65, 0x66, 0x74, 0x45, 0x6e, 0x64, - 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x52, 0x0a, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0a, 0x20, 0x03, 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, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x61, 0x67, - 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x61, - 0x6e, 0x67, 0x65, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x76, 0x0a, 0x0e, 0x52, - 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x15, 0x0a, - 0x11, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, - 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, - 0x4e, 0x10, 0x01, 0x12, 0x16, 0x0a, 0x12, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, - 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x10, 0x02, 0x12, 0x26, 0x0a, 0x22, 0x4c, - 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, - 0x4c, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x57, 0x5f, 0x50, 0x52, 0x45, 0x43, 0x45, 0x44, 0x49, 0x4e, - 0x47, 0x10, 0x03, 0x22, 0x7b, 0x0a, 0x0c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x6e, 0x64, 0x52, - 0x75, 0x6c, 0x65, 0x12, 0x13, 0x0a, 0x0f, 0x45, 0x4e, 0x44, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, - 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x52, 0x45, 0x41, - 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x01, 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, 0x02, 0x12, 0x29, 0x0a, 0x25, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, - 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x5f, 0x41, - 0x4c, 0x4c, 0x4f, 0x57, 0x5f, 0x46, 0x4f, 0x4c, 0x4c, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x03, - 0x22, 0xef, 0x05, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 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, 0x62, 0x0a, 0x0a, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 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, - 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x61, 0x67, 0x67, 0x72, - 0x65, 0x67, 0x61, 0x74, 0x65, 0x73, 0x12, 0x28, 0x0a, 0x10, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, - 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, - 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x62, 0x6f, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x62, - 0x6f, 0x1a, 0xe2, 0x01, 0x0a, 0x09, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, - 0x54, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x67, 0x67, 0x54, 0x79, 0x70, 0x65, 0x52, - 0x04, 0x74, 0x79, 0x70, 0x65, 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, 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, 0x1e, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, - 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x70, 0x65, 0x72, - 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x76, 0x67, 0x5f, 0x6d, - 0x65, 0x64, 0x69, 0x61, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x76, 0x67, - 0x4d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x22, 0xa5, 0x01, 0x0a, 0x07, 0x41, 0x67, 0x67, 0x54, 0x79, - 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x55, 0x4d, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x41, - 0x42, 0x53, 0x5f, 0x53, 0x55, 0x4d, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x47, 0x52, 0x4f, 0x55, - 0x50, 0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x56, 0x47, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, - 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x52, 0x53, 0x54, - 0x10, 0x05, 0x12, 0x08, 0x0a, 0x04, 0x4c, 0x41, 0x53, 0x54, 0x10, 0x06, 0x12, 0x07, 0x0a, 0x03, - 0x4d, 0x49, 0x4e, 0x10, 0x07, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x41, 0x58, 0x10, 0x08, 0x12, 0x0a, - 0x0a, 0x06, 0x4d, 0x45, 0x44, 0x49, 0x41, 0x4e, 0x10, 0x09, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, - 0x52, 0x43, 0x45, 0x4e, 0x54, 0x49, 0x4c, 0x45, 0x10, 0x0a, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x54, - 0x44, 0x10, 0x0b, 0x12, 0x07, 0x0a, 0x03, 0x56, 0x41, 0x52, 0x10, 0x0c, 0x12, 0x10, 0x0a, 0x0c, - 0x57, 0x45, 0x49, 0x47, 0x48, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x56, 0x47, 0x10, 0x0d, 0x3a, 0x02, - 0x18, 0x01, 0x22, 0x97, 0x02, 0x0a, 0x13, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, + 0x6e, 0x73, 0x12, 0x20, 0x0a, 0x0c, 0x61, 0x73, 0x5f, 0x6f, 0x66, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x61, 0x73, 0x4f, 0x66, 0x43, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, + 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x22, 0xb0, 0x01, 0x0a, 0x0e, 0x4d, + 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x4e, 0x0a, + 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 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, 0x28, 0x0a, + 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74, 0x63, + 0x68, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x24, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x41, 0x64, 0x64, 0x22, 0xbf, 0x01, + 0x0a, 0x16, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, + 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, 0x5d, 0x0a, 0x11, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x5f, 0x69, + 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 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, + 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x52, 0x0f, + 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x22, + 0xf8, 0x07, 0x0a, 0x16, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, + 0x6c, 0x65, 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, 0x3e, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, 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, 0x28, 0x0a, 0x10, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, - 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x67, 0x72, - 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xef, 0x1b, 0x0a, - 0x07, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x62, 0x73, 0x5f, - 0x73, 0x75, 0x6d, 0x18, 0x01, 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, 0x67, - 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x62, 0x73, - 0x53, 0x75, 0x6d, 0x48, 0x00, 0x52, 0x06, 0x61, 0x62, 0x73, 0x53, 0x75, 0x6d, 0x12, 0x80, 0x01, - 0x0a, 0x16, 0x61, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x5f, 0x70, 0x65, - 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x47, - 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, - 0x65, 0x63, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x50, 0x65, 0x72, - 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x15, 0x61, 0x70, 0x70, 0x72, 0x6f, - 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, - 0x12, 0x49, 0x0a, 0x03, 0x61, 0x76, 0x67, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, - 0x63, 0x41, 0x76, 0x67, 0x48, 0x00, 0x52, 0x03, 0x61, 0x76, 0x67, 0x12, 0x68, 0x0a, 0x0e, 0x63, - 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, - 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, - 0x69, 0x6e, 0x63, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x44, 0x69, 0x73, - 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x58, 0x0a, 0x08, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, - 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 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, 0x2e, 0x0a, 0x13, + 0x65, 0x78, 0x61, 0x63, 0x74, 0x5f, 0x6d, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x65, 0x78, 0x61, 0x63, 0x74, + 0x4d, 0x61, 0x74, 0x63, 0x68, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x2a, 0x0a, 0x11, + 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x65, 0x66, 0x74, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x72, 0x0a, 0x10, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x48, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x52, 0x61, + 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0e, 0x72, 0x61, + 0x6e, 0x67, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x2c, 0x0a, 0x12, + 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x63, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x72, 0x69, 0x67, 0x68, 0x74, 0x52, + 0x61, 0x6e, 0x67, 0x65, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x6c, 0x0a, 0x0e, 0x72, 0x61, + 0x6e, 0x67, 0x65, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x75, 0x6c, 0x65, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x0e, 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, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, + 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x52, 0x61, + 0x6e, 0x67, 0x65, 0x45, 0x6e, 0x64, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x0c, 0x72, 0x61, 0x6e, 0x67, + 0x65, 0x45, 0x6e, 0x64, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x6c, 0x65, 0x66, 0x74, + 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x6c, 0x65, 0x66, 0x74, 0x45, 0x6e, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, + 0x12, 0x52, 0x0a, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x0a, 0x20, 0x03, 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, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x6d, 0x61, + 0x74, 0x63, 0x68, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x61, 0x6e, 0x67, 0x65, + 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x76, 0x0a, 0x0e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, + 0x61, 0x72, 0x74, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x41, 0x52, 0x54, + 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, + 0x0a, 0x09, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x10, 0x01, 0x12, 0x16, 0x0a, + 0x12, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, + 0x55, 0x41, 0x4c, 0x10, 0x02, 0x12, 0x26, 0x0a, 0x22, 0x4c, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x48, + 0x41, 0x4e, 0x5f, 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, + 0x57, 0x5f, 0x50, 0x52, 0x45, 0x43, 0x45, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x22, 0x7b, 0x0a, + 0x0c, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x45, 0x6e, 0x64, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x13, 0x0a, + 0x0f, 0x45, 0x4e, 0x44, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, + 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, + 0x41, 0x4e, 0x10, 0x01, 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, 0x02, 0x12, + 0x29, 0x0a, 0x25, 0x47, 0x52, 0x45, 0x41, 0x54, 0x45, 0x52, 0x5f, 0x54, 0x48, 0x41, 0x4e, 0x5f, + 0x4f, 0x52, 0x5f, 0x45, 0x51, 0x55, 0x41, 0x4c, 0x5f, 0x41, 0x4c, 0x4c, 0x4f, 0x57, 0x5f, 0x46, + 0x4f, 0x4c, 0x4c, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x22, 0xef, 0x05, 0x0a, 0x15, 0x43, + 0x6f, 0x6d, 0x62, 0x6f, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 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, 0x41, 0x67, 0x67, 0x53, - 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x44, 0x69, 0x73, 0x74, 0x69, - 0x6e, 0x63, 0x74, 0x48, 0x00, 0x52, 0x08, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, - 0x4f, 0x0a, 0x05, 0x66, 0x69, 0x72, 0x73, 0x74, 0x18, 0x06, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, - 0x65, 0x63, 0x46, 0x69, 0x72, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x66, 0x69, 0x72, 0x73, 0x74, - 0x12, 0x55, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x07, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, - 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x07, - 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x52, 0x0a, 0x06, 0x66, 0x72, 0x65, 0x65, 0x7a, - 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 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, 0x62, 0x0a, 0x0a, + 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x42, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x73, + 0x12, 0x28, 0x0a, 0x10, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, + 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x6f, + 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x62, 0x6f, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0a, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x62, 0x6f, 0x1a, 0xe2, 0x01, 0x0a, 0x09, + 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x12, 0x54, 0x0a, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x53, - 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x72, 0x65, 0x65, 0x7a, - 0x65, 0x48, 0x00, 0x52, 0x06, 0x66, 0x72, 0x65, 0x65, 0x7a, 0x65, 0x12, 0x4f, 0x0a, 0x05, 0x67, - 0x72, 0x6f, 0x75, 0x70, 0x18, 0x09, 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, 0x41, - 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x48, 0x00, 0x52, 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x4c, 0x0a, 0x04, - 0x6c, 0x61, 0x73, 0x74, 0x18, 0x0a, 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, 0x41, - 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4c, 0x61, - 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x6c, 0x61, 0x73, 0x74, 0x12, 0x49, 0x0a, 0x03, 0x6d, 0x61, - 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, + 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, 0x2e, 0x41, 0x67, 0x67, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 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, + 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, 0x1e, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, + 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x76, 0x67, 0x5f, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x76, 0x67, 0x4d, 0x65, 0x64, 0x69, 0x61, 0x6e, + 0x22, 0xa5, 0x01, 0x0a, 0x07, 0x41, 0x67, 0x67, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, + 0x53, 0x55, 0x4d, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x41, 0x42, 0x53, 0x5f, 0x53, 0x55, 0x4d, + 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x10, 0x02, 0x12, 0x07, 0x0a, + 0x03, 0x41, 0x56, 0x47, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, + 0x04, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x49, 0x52, 0x53, 0x54, 0x10, 0x05, 0x12, 0x08, 0x0a, 0x04, + 0x4c, 0x41, 0x53, 0x54, 0x10, 0x06, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x49, 0x4e, 0x10, 0x07, 0x12, + 0x07, 0x0a, 0x03, 0x4d, 0x41, 0x58, 0x10, 0x08, 0x12, 0x0a, 0x0a, 0x06, 0x4d, 0x45, 0x44, 0x49, + 0x41, 0x4e, 0x10, 0x09, 0x12, 0x0e, 0x0a, 0x0a, 0x50, 0x45, 0x52, 0x43, 0x45, 0x4e, 0x54, 0x49, + 0x4c, 0x45, 0x10, 0x0a, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x54, 0x44, 0x10, 0x0b, 0x12, 0x07, 0x0a, + 0x03, 0x56, 0x41, 0x52, 0x10, 0x0c, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x45, 0x49, 0x47, 0x48, 0x54, + 0x45, 0x44, 0x5f, 0x41, 0x56, 0x47, 0x10, 0x0d, 0x3a, 0x02, 0x18, 0x01, 0x22, 0x97, 0x02, 0x0a, + 0x13, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x6c, 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, 0x41, 0x67, 0x67, 0x53, - 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x61, 0x78, 0x48, 0x00, - 0x52, 0x03, 0x6d, 0x61, 0x78, 0x12, 0x52, 0x0a, 0x06, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x18, - 0x0c, 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, 0x67, 0x67, 0x53, 0x70, 0x65, - 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x48, - 0x00, 0x52, 0x06, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x12, 0x49, 0x0a, 0x03, 0x6d, 0x69, 0x6e, - 0x18, 0x0d, 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, 0x67, 0x67, 0x53, 0x70, - 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x69, 0x6e, 0x48, 0x00, 0x52, - 0x03, 0x6d, 0x69, 0x6e, 0x12, 0x5e, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, - 0x6c, 0x65, 0x18, 0x0e, 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, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x50, 0x65, 0x72, 0x63, - 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, - 0x74, 0x69, 0x6c, 0x65, 0x12, 0x5d, 0x0a, 0x0c, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x66, - 0x69, 0x72, 0x73, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 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, 0x3e, 0x0a, 0x04, + 0x73, 0x70, 0x65, 0x63, 0x18, 0x03, 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, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, - 0x72, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x46, 0x69, - 0x72, 0x73, 0x74, 0x12, 0x5b, 0x0a, 0x0b, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x6c, 0x61, - 0x73, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x28, 0x0a, 0x10, + 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, + 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x22, 0xef, 0x1b, 0x0a, 0x07, 0x41, 0x67, 0x67, 0x53, 0x70, + 0x65, 0x63, 0x12, 0x53, 0x0a, 0x07, 0x61, 0x62, 0x73, 0x5f, 0x73, 0x75, 0x6d, 0x18, 0x01, 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, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, + 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x62, 0x73, 0x53, 0x75, 0x6d, 0x48, 0x00, 0x52, + 0x06, 0x61, 0x62, 0x73, 0x53, 0x75, 0x6d, 0x12, 0x80, 0x01, 0x0a, 0x16, 0x61, 0x70, 0x70, 0x72, + 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, + 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x47, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x4c, 0x61, 0x73, 0x74, - 0x12, 0x49, 0x0a, 0x03, 0x73, 0x74, 0x64, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, + 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x70, 0x70, 0x72, + 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, + 0x65, 0x48, 0x00, 0x52, 0x15, 0x61, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, + 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x49, 0x0a, 0x03, 0x61, 0x76, + 0x67, 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, 0x41, 0x67, 0x67, 0x53, + 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x76, 0x67, 0x48, 0x00, + 0x52, 0x03, 0x61, 0x76, 0x67, 0x12, 0x68, 0x0a, 0x0e, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x64, + 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, - 0x63, 0x53, 0x74, 0x64, 0x48, 0x00, 0x52, 0x03, 0x73, 0x74, 0x64, 0x12, 0x49, 0x0a, 0x03, 0x73, - 0x75, 0x6d, 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, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x75, 0x6d, 0x48, - 0x00, 0x52, 0x03, 0x73, 0x75, 0x6d, 0x12, 0x56, 0x0a, 0x08, 0x74, 0x5f, 0x64, 0x69, 0x67, 0x65, - 0x73, 0x74, 0x18, 0x13, 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, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x54, 0x44, 0x69, 0x67, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x74, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x52, - 0x0a, 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x18, 0x14, 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, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, - 0x65, 0x63, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x48, 0x00, 0x52, 0x06, 0x75, 0x6e, 0x69, 0x71, - 0x75, 0x65, 0x12, 0x5f, 0x0a, 0x0c, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x5f, 0x61, - 0x76, 0x67, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x63, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x48, 0x00, + 0x52, 0x0d, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, + 0x58, 0x0a, 0x08, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x18, 0x05, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, + 0x67, 0x53, 0x70, 0x65, 0x63, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x48, 0x00, 0x52, + 0x08, 0x64, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x4f, 0x0a, 0x05, 0x66, 0x69, 0x72, + 0x73, 0x74, 0x18, 0x06, 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, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x57, 0x65, 0x69, 0x67, - 0x68, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x0b, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, - 0x41, 0x76, 0x67, 0x12, 0x5f, 0x0a, 0x0c, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x5f, - 0x73, 0x75, 0x6d, 0x18, 0x16, 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, 0x41, 0x67, - 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x57, 0x65, 0x69, - 0x67, 0x68, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x0b, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, - 0x64, 0x53, 0x75, 0x6d, 0x12, 0x49, 0x0a, 0x03, 0x76, 0x61, 0x72, 0x18, 0x17, 0x20, 0x01, 0x28, + 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x69, 0x72, 0x73, + 0x74, 0x48, 0x00, 0x52, 0x05, 0x66, 0x69, 0x72, 0x73, 0x74, 0x12, 0x55, 0x0a, 0x07, 0x66, 0x6f, + 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x07, 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, + 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, + 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x48, 0x00, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, + 0x61, 0x12, 0x52, 0x0a, 0x06, 0x66, 0x72, 0x65, 0x65, 0x7a, 0x65, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, + 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x72, 0x65, 0x65, 0x7a, 0x65, 0x48, 0x00, 0x52, 0x06, 0x66, + 0x72, 0x65, 0x65, 0x7a, 0x65, 0x12, 0x4f, 0x0a, 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x09, + 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, + 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x48, 0x00, 0x52, + 0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x4c, 0x0a, 0x04, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x0a, + 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, + 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4c, 0x61, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, + 0x6c, 0x61, 0x73, 0x74, 0x12, 0x49, 0x0a, 0x03, 0x6d, 0x61, 0x78, 0x18, 0x0b, 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, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, - 0x67, 0x53, 0x70, 0x65, 0x63, 0x56, 0x61, 0x72, 0x48, 0x00, 0x52, 0x03, 0x76, 0x61, 0x72, 0x1a, - 0x75, 0x0a, 0x1c, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, - 0x69, 0x6d, 0x61, 0x74, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, - 0x1e, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x01, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, - 0x25, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, - 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0x37, 0x0a, 0x14, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, - 0x63, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x1f, - 0x0a, 0x0b, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x4e, 0x75, 0x6c, 0x6c, 0x73, 0x1a, - 0x36, 0x0a, 0x0f, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, - 0x63, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6e, 0x75, - 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, - 0x64, 0x65, 0x4e, 0x75, 0x6c, 0x6c, 0x73, 0x1a, 0x4b, 0x0a, 0x0e, 0x41, 0x67, 0x67, 0x53, 0x70, - 0x65, 0x63, 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x6f, 0x72, - 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6d, - 0x75, 0x6c, 0x61, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x5f, 0x74, 0x6f, 0x6b, - 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x54, - 0x6f, 0x6b, 0x65, 0x6e, 0x1a, 0x45, 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, - 0x65, 0x64, 0x69, 0x61, 0x6e, 0x12, 0x34, 0x0a, 0x16, 0x61, 0x76, 0x65, 0x72, 0x61, 0x67, 0x65, - 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x6c, 0x79, 0x5f, 0x64, 0x69, 0x76, 0x69, 0x64, 0x65, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x61, 0x76, 0x65, 0x72, 0x61, 0x67, 0x65, 0x45, 0x76, - 0x65, 0x6e, 0x6c, 0x79, 0x44, 0x69, 0x76, 0x69, 0x64, 0x65, 0x64, 0x1a, 0x69, 0x0a, 0x11, 0x41, - 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, - 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, - 0x12, 0x34, 0x0a, 0x16, 0x61, 0x76, 0x65, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x65, 0x76, 0x65, 0x6e, - 0x6c, 0x79, 0x5f, 0x64, 0x69, 0x76, 0x69, 0x64, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x14, 0x61, 0x76, 0x65, 0x72, 0x61, 0x67, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x6c, 0x79, 0x44, - 0x69, 0x76, 0x69, 0x64, 0x65, 0x64, 0x1a, 0x69, 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, - 0x63, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x12, 0x58, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 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, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, - 0x73, 0x1a, 0x36, 0x0a, 0x13, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, - 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 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, 0x1a, 0x47, 0x0a, 0x0e, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x54, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0b, 0x63, - 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, - 0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x88, - 0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x1a, 0xa9, 0x01, 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x55, 0x6e, - 0x69, 0x71, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, - 0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, - 0x6c, 0x75, 0x64, 0x65, 0x4e, 0x75, 0x6c, 0x6c, 0x73, 0x12, 0x73, 0x0a, 0x13, 0x6e, 0x6f, 0x6e, - 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6c, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, - 0x68, 0x61, 0x76, 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, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4e, 0x6f, 0x6e, 0x55, 0x6e, 0x69, - 0x71, 0x75, 0x65, 0x53, 0x65, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x52, 0x11, 0x6e, 0x6f, 0x6e, - 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x53, 0x65, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x1a, 0xa8, - 0x03, 0x0a, 0x18, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4e, 0x6f, 0x6e, 0x55, 0x6e, 0x69, - 0x71, 0x75, 0x65, 0x53, 0x65, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x12, 0x4d, 0x0a, 0x0a, 0x6e, - 0x75, 0x6c, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, + 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x61, 0x78, 0x48, 0x00, 0x52, 0x03, 0x6d, 0x61, 0x78, 0x12, + 0x52, 0x0a, 0x06, 0x6d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x18, 0x0c, 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, 0x4e, 0x75, 0x6c, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, - 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, - 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, - 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x11, 0x48, 0x00, 0x52, 0x08, 0x69, 0x6e, 0x74, 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, 0x21, 0x0a, 0x0b, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x5f, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x02, 0x48, 0x00, 0x52, 0x0a, 0x66, 0x6c, 0x6f, 0x61, - 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, - 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x06, 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, 0x07, 0x20, 0x01, 0x28, 0x08, 0x48, - 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, - 0x62, 0x79, 0x74, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x11, - 0x48, 0x00, 0x52, 0x09, 0x62, 0x79, 0x74, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, - 0x0b, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x09, 0x20, 0x01, - 0x28, 0x11, 0x48, 0x00, 0x52, 0x0a, 0x73, 0x68, 0x6f, 0x72, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, - 0x12, 0x1f, 0x0a, 0x0a, 0x63, 0x68, 0x61, 0x72, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x0a, - 0x20, 0x01, 0x28, 0x11, 0x48, 0x00, 0x52, 0x09, 0x63, 0x68, 0x61, 0x72, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x1a, 0x36, 0x0a, 0x0f, 0x41, 0x67, 0x67, - 0x53, 0x70, 0x65, 0x63, 0x57, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x12, 0x23, 0x0a, 0x0d, - 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, - 0x6e, 0x1a, 0x0f, 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x62, 0x73, 0x53, - 0x75, 0x6d, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x76, 0x67, - 0x1a, 0x0e, 0x0a, 0x0c, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x69, 0x72, 0x73, 0x74, - 0x1a, 0x0f, 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x72, 0x65, 0x65, 0x7a, - 0x65, 0x1a, 0x0e, 0x0a, 0x0c, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x47, 0x72, 0x6f, 0x75, - 0x70, 0x1a, 0x0d, 0x0a, 0x0b, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4c, 0x61, 0x73, 0x74, - 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x61, 0x78, 0x1a, 0x0c, - 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x69, 0x6e, 0x1a, 0x0c, 0x0a, 0x0a, - 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x74, 0x64, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, - 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x75, 0x6d, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, - 0x70, 0x65, 0x63, 0x56, 0x61, 0x72, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xae, - 0x03, 0x0a, 0x10, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 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, 0x5d, 0x0a, 0x11, 0x69, - 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 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, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, - 0x61, 0x6c, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0d, 0x70, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x45, 0x6d, 0x70, 0x74, - 0x79, 0x12, 0x52, 0x0a, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x18, 0x05, 0x20, 0x03, 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, 0x41, 0x67, 0x67, 0x72, - 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x61, 0x67, 0x67, 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, - 0xd4, 0x06, 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, + 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, + 0x70, 0x65, 0x63, 0x4d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x48, 0x00, 0x52, 0x06, 0x6d, 0x65, 0x64, + 0x69, 0x61, 0x6e, 0x12, 0x49, 0x0a, 0x03, 0x6d, 0x69, 0x6e, 0x18, 0x0d, 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, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6c, 0x75, - 0x6d, 0x6e, 0x73, 0x48, 0x00, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x57, - 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 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, 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, 0x48, 0x00, - 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x66, 0x0a, 0x0d, 0x66, 0x69, 0x72, 0x73, 0x74, - 0x5f, 0x72, 0x6f, 0x77, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x69, 0x6f, 0x6e, 0x2e, 0x41, - 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, - 0x48, 0x00, 0x52, 0x0b, 0x66, 0x69, 0x72, 0x73, 0x74, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, - 0x64, 0x0a, 0x0c, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x5f, 0x6b, 0x65, 0x79, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x48, 0x00, 0x52, 0x0a, 0x6c, 0x61, 0x73, 0x74, 0x52, - 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, 0x63, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, + 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, + 0x53, 0x70, 0x65, 0x63, 0x4d, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x6d, 0x69, 0x6e, 0x12, 0x5e, + 0x0a, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x0e, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, + 0x48, 0x00, 0x52, 0x0a, 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x5d, + 0x0a, 0x0c, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x66, 0x69, 0x72, 0x73, 0x74, 0x18, 0x0f, + 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, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, + 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x48, 0x00, + 0x52, 0x0b, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x46, 0x69, 0x72, 0x73, 0x74, 0x12, 0x5b, 0x0a, + 0x0b, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x10, 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, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x0a, + 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x4c, 0x61, 0x73, 0x74, 0x12, 0x49, 0x0a, 0x03, 0x73, 0x74, + 0x64, 0x18, 0x11, 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, 0x67, 0x67, 0x53, + 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x74, 0x64, 0x48, 0x00, + 0x52, 0x03, 0x73, 0x74, 0x64, 0x12, 0x49, 0x0a, 0x03, 0x73, 0x75, 0x6d, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x75, 0x6d, 0x48, 0x00, 0x52, 0x03, 0x73, 0x75, 0x6d, + 0x12, 0x56, 0x0a, 0x08, 0x74, 0x5f, 0x64, 0x69, 0x67, 0x65, 0x73, 0x74, 0x18, 0x13, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x54, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, + 0x07, 0x74, 0x44, 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x52, 0x0a, 0x06, 0x75, 0x6e, 0x69, 0x71, + 0x75, 0x65, 0x18, 0x14, 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, 0x67, 0x67, - 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, - 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 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, 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, + 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x55, 0x6e, 0x69, 0x71, + 0x75, 0x65, 0x48, 0x00, 0x52, 0x06, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x12, 0x5f, 0x0a, 0x0c, + 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x76, 0x67, 0x18, 0x15, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x57, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x48, 0x00, + 0x52, 0x0b, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x41, 0x76, 0x67, 0x12, 0x5f, 0x0a, + 0x0c, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x5f, 0x73, 0x75, 0x6d, 0x18, 0x16, 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, 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, + 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, + 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x57, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x48, + 0x00, 0x52, 0x0b, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x53, 0x75, 0x6d, 0x12, 0x49, + 0x0a, 0x03, 0x76, 0x61, 0x72, 0x18, 0x17, 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, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x56, + 0x61, 0x72, 0x48, 0x00, 0x52, 0x03, 0x76, 0x61, 0x72, 0x1a, 0x75, 0x0a, 0x1c, 0x41, 0x67, 0x67, + 0x53, 0x70, 0x65, 0x63, 0x41, 0x70, 0x70, 0x72, 0x6f, 0x78, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x50, + 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x65, 0x72, + 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, 0x70, + 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x25, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, + 0x52, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, + 0x42, 0x0e, 0x0a, 0x0c, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x1a, 0x37, 0x0a, 0x14, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x43, 0x6f, 0x75, 0x6e, 0x74, + 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x4e, 0x75, 0x6c, 0x6c, 0x73, 0x1a, 0x36, 0x0a, 0x0f, 0x41, 0x67, 0x67, + 0x53, 0x70, 0x65, 0x63, 0x44, 0x69, 0x73, 0x74, 0x69, 0x6e, 0x63, 0x74, 0x12, 0x23, 0x0a, 0x0d, + 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4e, 0x75, 0x6c, 0x6c, + 0x73, 0x1a, 0x4b, 0x0a, 0x0e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x6f, 0x72, 0x6d, + 0x75, 0x6c, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x1f, 0x0a, + 0x0b, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x1a, 0x45, + 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x65, 0x64, 0x69, 0x61, 0x6e, 0x12, + 0x34, 0x0a, 0x16, 0x61, 0x76, 0x65, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x6c, + 0x79, 0x5f, 0x64, 0x69, 0x76, 0x69, 0x64, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x14, 0x61, 0x76, 0x65, 0x72, 0x61, 0x67, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x6c, 0x79, 0x44, 0x69, + 0x76, 0x69, 0x64, 0x65, 0x64, 0x1a, 0x69, 0x0a, 0x11, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, + 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x65, + 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0a, + 0x70, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x69, 0x6c, 0x65, 0x12, 0x34, 0x0a, 0x16, 0x61, 0x76, + 0x65, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x6c, 0x79, 0x5f, 0x64, 0x69, 0x76, + 0x69, 0x64, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x61, 0x76, 0x65, 0x72, + 0x61, 0x67, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x6c, 0x79, 0x44, 0x69, 0x76, 0x69, 0x64, 0x65, 0x64, + 0x1a, 0x69, 0x0a, 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x12, 0x58, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 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, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x2e, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 0x6e, 0x52, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x36, 0x0a, 0x13, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x75, + 0x6d, 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, 0x1a, 0x47, 0x0a, 0x0e, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x54, 0x44, + 0x69, 0x67, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x63, 0x6f, + 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x88, 0x01, 0x01, 0x42, 0x0e, 0x0a, 0x0c, + 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0xa9, 0x01, 0x0a, + 0x0d, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x12, 0x23, + 0x0a, 0x0d, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x5f, 0x6e, 0x75, 0x6c, 0x6c, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4e, 0x75, + 0x6c, 0x6c, 0x73, 0x12, 0x73, 0x0a, 0x13, 0x6e, 0x6f, 0x6e, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, + 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x43, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x41, 0x67, 0x67, + 0x53, 0x70, 0x65, 0x63, 0x4e, 0x6f, 0x6e, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x53, 0x65, 0x6e, + 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x52, 0x11, 0x6e, 0x6f, 0x6e, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, + 0x53, 0x65, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x1a, 0xa8, 0x03, 0x0a, 0x18, 0x41, 0x67, 0x67, + 0x53, 0x70, 0x65, 0x63, 0x4e, 0x6f, 0x6e, 0x55, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x53, 0x65, 0x6e, + 0x74, 0x69, 0x6e, 0x65, 0x6c, 0x12, 0x4d, 0x0a, 0x0a, 0x6e, 0x75, 0x6c, 0x6c, 0x5f, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x01, 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, 0x4e, 0x75, + 0x6c, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x09, 0x6e, 0x75, 0x6c, 0x6c, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, + 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1d, 0x0a, 0x09, 0x69, 0x6e, 0x74, + 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x11, 0x48, 0x00, 0x52, 0x08, + 0x69, 0x6e, 0x74, 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, 0x21, 0x0a, + 0x0b, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x02, 0x48, 0x00, 0x52, 0x0a, 0x66, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x06, 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, 0x07, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f, 0x6f, + 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x79, 0x74, 0x65, 0x5f, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x11, 0x48, 0x00, 0x52, 0x09, 0x62, 0x79, + 0x74, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x21, 0x0a, 0x0b, 0x73, 0x68, 0x6f, 0x72, 0x74, + 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x11, 0x48, 0x00, 0x52, 0x0a, + 0x73, 0x68, 0x6f, 0x72, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x63, 0x68, + 0x61, 0x72, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x11, 0x48, 0x00, + 0x52, 0x09, 0x63, 0x68, 0x61, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x1a, 0x36, 0x0a, 0x0f, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x57, 0x65, + 0x69, 0x67, 0x68, 0x74, 0x65, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x77, 0x65, 0x69, 0x67, 0x68, 0x74, + 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x77, + 0x65, 0x69, 0x67, 0x68, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0x0f, 0x0a, 0x0d, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x62, 0x73, 0x53, 0x75, 0x6d, 0x1a, 0x0c, 0x0a, 0x0a, + 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x41, 0x76, 0x67, 0x1a, 0x0e, 0x0a, 0x0c, 0x41, 0x67, + 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x69, 0x72, 0x73, 0x74, 0x1a, 0x0f, 0x0a, 0x0d, 0x41, 0x67, + 0x67, 0x53, 0x70, 0x65, 0x63, 0x46, 0x72, 0x65, 0x65, 0x7a, 0x65, 0x1a, 0x0e, 0x0a, 0x0c, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x1a, 0x0d, 0x0a, 0x0b, 0x41, + 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x4c, 0x61, 0x73, 0x74, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, + 0x67, 0x53, 0x70, 0x65, 0x63, 0x4d, 0x61, 0x78, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, + 0x70, 0x65, 0x63, 0x4d, 0x69, 0x6e, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, + 0x63, 0x53, 0x74, 0x64, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x53, + 0x75, 0x6d, 0x1a, 0x0c, 0x0a, 0x0a, 0x41, 0x67, 0x67, 0x53, 0x70, 0x65, 0x63, 0x56, 0x61, 0x72, + 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0xae, 0x03, 0x0a, 0x10, 0x41, 0x67, 0x67, + 0x72, 0x65, 0x67, 0x61, 0x74, 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, 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, + 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, 0x5d, 0x0a, 0x11, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, + 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 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, 0x0f, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x61, 0x6c, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x73, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, + 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x72, 0x76, 0x65, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x52, 0x0a, 0x0c, 0x61, + 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 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, 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, + 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0c, 0x61, 0x67, 0x67, 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, 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, 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, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x48, 0x00, 0x52, + 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x57, 0x0a, 0x05, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 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, 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, + 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, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x12, 0x66, 0x0a, 0x0d, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x5f, 0x6b, + 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 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, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, + 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, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x48, 0x00, 0x52, 0x0b, 0x66, 0x69, + 0x72, 0x73, 0x74, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, 0x64, 0x0a, 0x0c, 0x6c, 0x61, 0x73, + 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 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, 0x69, 0x6f, 0x6e, 0x2e, + 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, + 0x79, 0x48, 0x00, 0x52, 0x0a, 0x6c, 0x61, 0x73, 0x74, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, + 0x63, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, + 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, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5d, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x18, + 0x06, 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, 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, + 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, 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, 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, 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, + 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, 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, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, + 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, - 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, + 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, - 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, + 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, 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, + 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, 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, + 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, 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, 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, + 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, 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, + 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, 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, 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, + 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, - 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, + 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, 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, + 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, 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, 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, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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, + 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, 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, 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, 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, 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, + 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, 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, 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, + 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, 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, + 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, 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, + 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, 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, 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, + 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, 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, + 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, 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, 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, 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, + 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, 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, + 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, 0x70, 0x68, 0x61, 0x76, 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, 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, + 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, 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, + 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, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 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, 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, - 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, + 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, 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, + 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, 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, 0x68, 0x61, 0x76, 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, + 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, 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, 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, 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, - 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, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, - 0x61, 0x76, 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, + 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, 0x45, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, + 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, 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, 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, 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, + 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, 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, + 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, 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, + 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, 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, + 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, 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, 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, + 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, 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, 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, + 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, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, + 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, 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, + 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, 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, 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, + 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, 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, 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, + 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, 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, 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, 0x84, 0x01, 0x0a, 0x04, 0x56, 0x69, 0x65, 0x77, 0x12, 0x38, 0x2e, 0x69, + 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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, 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 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, + 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, 0x82, 0x01, 0x0a, 0x08, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x12, 0x32, 0x2e, + 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, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, + 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, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, + 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, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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, 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, 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, + 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, 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, + 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, 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, 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, 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, - 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, 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, + 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, 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, 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, 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, + 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, 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 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, + 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, 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, 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, + 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, 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, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, - 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, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, + 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, 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, + 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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, + 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, 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, + 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, 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, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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, + 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, 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, 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, + 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, 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, 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, + 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, 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, 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, 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, + 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, 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, - 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, + 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, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 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, 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, + 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, } var ( @@ -12516,7 +12663,7 @@ func file_deephaven_proto_table_proto_rawDescGZIP() []byte { } var file_deephaven_proto_table_proto_enumTypes = make([]protoimpl.EnumInfo, 12) -var file_deephaven_proto_table_proto_msgTypes = make([]protoimpl.MessageInfo, 125) +var file_deephaven_proto_table_proto_msgTypes = make([]protoimpl.MessageInfo, 127) var file_deephaven_proto_table_proto_goTypes = []interface{}{ (BadDataBehavior)(0), // 0: io.deephaven.proto.backplane.grpc.BadDataBehavior (UpdateByNullBehavior)(0), // 1: io.deephaven.proto.backplane.grpc.UpdateByNullBehavior @@ -12539,497 +12686,501 @@ var file_deephaven_proto_table_proto_goTypes = []interface{}{ (*EmptyTableRequest)(nil), // 18: io.deephaven.proto.backplane.grpc.EmptyTableRequest (*TimeTableRequest)(nil), // 19: io.deephaven.proto.backplane.grpc.TimeTableRequest (*SelectOrUpdateRequest)(nil), // 20: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - (*MathContext)(nil), // 21: io.deephaven.proto.backplane.grpc.MathContext - (*UpdateByWindowScale)(nil), // 22: io.deephaven.proto.backplane.grpc.UpdateByWindowScale - (*UpdateByEmOptions)(nil), // 23: io.deephaven.proto.backplane.grpc.UpdateByEmOptions - (*UpdateByDeltaOptions)(nil), // 24: io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions - (*UpdateByRequest)(nil), // 25: io.deephaven.proto.backplane.grpc.UpdateByRequest - (*SelectDistinctRequest)(nil), // 26: io.deephaven.proto.backplane.grpc.SelectDistinctRequest - (*DropColumnsRequest)(nil), // 27: io.deephaven.proto.backplane.grpc.DropColumnsRequest - (*UnstructuredFilterTableRequest)(nil), // 28: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest - (*HeadOrTailRequest)(nil), // 29: io.deephaven.proto.backplane.grpc.HeadOrTailRequest - (*HeadOrTailByRequest)(nil), // 30: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - (*UngroupRequest)(nil), // 31: io.deephaven.proto.backplane.grpc.UngroupRequest - (*MergeTablesRequest)(nil), // 32: io.deephaven.proto.backplane.grpc.MergeTablesRequest - (*SnapshotTableRequest)(nil), // 33: io.deephaven.proto.backplane.grpc.SnapshotTableRequest - (*SnapshotWhenTableRequest)(nil), // 34: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest - (*CrossJoinTablesRequest)(nil), // 35: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest - (*NaturalJoinTablesRequest)(nil), // 36: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest - (*ExactJoinTablesRequest)(nil), // 37: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest - (*LeftJoinTablesRequest)(nil), // 38: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest - (*AsOfJoinTablesRequest)(nil), // 39: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest - (*AjRajTablesRequest)(nil), // 40: io.deephaven.proto.backplane.grpc.AjRajTablesRequest - (*MultiJoinInput)(nil), // 41: io.deephaven.proto.backplane.grpc.MultiJoinInput - (*MultiJoinTablesRequest)(nil), // 42: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest - (*RangeJoinTablesRequest)(nil), // 43: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest - (*ComboAggregateRequest)(nil), // 44: io.deephaven.proto.backplane.grpc.ComboAggregateRequest - (*AggregateAllRequest)(nil), // 45: io.deephaven.proto.backplane.grpc.AggregateAllRequest - (*AggSpec)(nil), // 46: io.deephaven.proto.backplane.grpc.AggSpec - (*AggregateRequest)(nil), // 47: io.deephaven.proto.backplane.grpc.AggregateRequest - (*Aggregation)(nil), // 48: io.deephaven.proto.backplane.grpc.Aggregation - (*SortDescriptor)(nil), // 49: io.deephaven.proto.backplane.grpc.SortDescriptor - (*SortTableRequest)(nil), // 50: io.deephaven.proto.backplane.grpc.SortTableRequest - (*FilterTableRequest)(nil), // 51: io.deephaven.proto.backplane.grpc.FilterTableRequest - (*SeekRowRequest)(nil), // 52: io.deephaven.proto.backplane.grpc.SeekRowRequest - (*SeekRowResponse)(nil), // 53: io.deephaven.proto.backplane.grpc.SeekRowResponse - (*Reference)(nil), // 54: io.deephaven.proto.backplane.grpc.Reference - (*Literal)(nil), // 55: io.deephaven.proto.backplane.grpc.Literal - (*Value)(nil), // 56: io.deephaven.proto.backplane.grpc.Value - (*Condition)(nil), // 57: io.deephaven.proto.backplane.grpc.Condition - (*AndCondition)(nil), // 58: io.deephaven.proto.backplane.grpc.AndCondition - (*OrCondition)(nil), // 59: io.deephaven.proto.backplane.grpc.OrCondition - (*NotCondition)(nil), // 60: io.deephaven.proto.backplane.grpc.NotCondition - (*CompareCondition)(nil), // 61: io.deephaven.proto.backplane.grpc.CompareCondition - (*InCondition)(nil), // 62: io.deephaven.proto.backplane.grpc.InCondition - (*InvokeCondition)(nil), // 63: io.deephaven.proto.backplane.grpc.InvokeCondition - (*IsNullCondition)(nil), // 64: io.deephaven.proto.backplane.grpc.IsNullCondition - (*MatchesCondition)(nil), // 65: io.deephaven.proto.backplane.grpc.MatchesCondition - (*ContainsCondition)(nil), // 66: io.deephaven.proto.backplane.grpc.ContainsCondition - (*SearchCondition)(nil), // 67: io.deephaven.proto.backplane.grpc.SearchCondition - (*FlattenRequest)(nil), // 68: io.deephaven.proto.backplane.grpc.FlattenRequest - (*MetaTableRequest)(nil), // 69: io.deephaven.proto.backplane.grpc.MetaTableRequest - (*RunChartDownsampleRequest)(nil), // 70: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest - (*CreateInputTableRequest)(nil), // 71: io.deephaven.proto.backplane.grpc.CreateInputTableRequest - (*WhereInRequest)(nil), // 72: io.deephaven.proto.backplane.grpc.WhereInRequest - (*ColumnStatisticsRequest)(nil), // 73: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest - (*SliceRequest)(nil), // 74: io.deephaven.proto.backplane.grpc.SliceRequest - (*BatchTableRequest)(nil), // 75: io.deephaven.proto.backplane.grpc.BatchTableRequest - (*UpdateByWindowScale_UpdateByWindowTicks)(nil), // 76: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicks - (*UpdateByWindowScale_UpdateByWindowTime)(nil), // 77: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTime - (*UpdateByRequest_UpdateByOptions)(nil), // 78: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions - (*UpdateByRequest_UpdateByOperation)(nil), // 79: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation - (*UpdateByRequest_UpdateByOperation_UpdateByColumn)(nil), // 80: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec)(nil), // 81: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum)(nil), // 82: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSum - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin)(nil), // 83: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMin - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax)(nil), // 84: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct)(nil), // 85: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProduct - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill)(nil), // 86: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFill - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma)(nil), // 87: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms)(nil), // 88: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin)(nil), // 89: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax)(nil), // 90: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd)(nil), // 91: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta)(nil), // 92: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum)(nil), // 93: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup)(nil), // 94: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg)(nil), // 95: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin)(nil), // 96: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax)(nil), // 97: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct)(nil), // 98: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount)(nil), // 99: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd)(nil), // 100: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg)(nil), // 101: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg - (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula)(nil), // 102: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula - (*ComboAggregateRequest_Aggregate)(nil), // 103: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate - (*AggSpec_AggSpecApproximatePercentile)(nil), // 104: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentile - (*AggSpec_AggSpecCountDistinct)(nil), // 105: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinct - (*AggSpec_AggSpecDistinct)(nil), // 106: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinct - (*AggSpec_AggSpecFormula)(nil), // 107: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormula - (*AggSpec_AggSpecMedian)(nil), // 108: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedian - (*AggSpec_AggSpecPercentile)(nil), // 109: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentile - (*AggSpec_AggSpecSorted)(nil), // 110: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted - (*AggSpec_AggSpecSortedColumn)(nil), // 111: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn - (*AggSpec_AggSpecTDigest)(nil), // 112: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigest - (*AggSpec_AggSpecUnique)(nil), // 113: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique - (*AggSpec_AggSpecNonUniqueSentinel)(nil), // 114: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel - (*AggSpec_AggSpecWeighted)(nil), // 115: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted - (*AggSpec_AggSpecAbsSum)(nil), // 116: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum - (*AggSpec_AggSpecAvg)(nil), // 117: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvg - (*AggSpec_AggSpecFirst)(nil), // 118: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirst - (*AggSpec_AggSpecFreeze)(nil), // 119: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreeze - (*AggSpec_AggSpecGroup)(nil), // 120: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroup - (*AggSpec_AggSpecLast)(nil), // 121: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLast - (*AggSpec_AggSpecMax)(nil), // 122: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMax - (*AggSpec_AggSpecMin)(nil), // 123: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMin - (*AggSpec_AggSpecStd)(nil), // 124: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStd - (*AggSpec_AggSpecSum)(nil), // 125: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSum - (*AggSpec_AggSpecVar)(nil), // 126: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVar - (*Aggregation_AggregationColumns)(nil), // 127: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns - (*Aggregation_AggregationCount)(nil), // 128: io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount - (*Aggregation_AggregationRowKey)(nil), // 129: io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey - (*Aggregation_AggregationPartition)(nil), // 130: io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition - (*RunChartDownsampleRequest_ZoomRange)(nil), // 131: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange - (*CreateInputTableRequest_InputTableKind)(nil), // 132: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind - (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly)(nil), // 133: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly - (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked)(nil), // 134: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked - (*CreateInputTableRequest_InputTableKind_Blink)(nil), // 135: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink - (*BatchTableRequest_Operation)(nil), // 136: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation - (*ticket.Ticket)(nil), // 137: io.deephaven.proto.backplane.grpc.Ticket + (*Selectable)(nil), // 21: io.deephaven.proto.backplane.grpc.Selectable + (*MathContext)(nil), // 22: io.deephaven.proto.backplane.grpc.MathContext + (*UpdateByWindowScale)(nil), // 23: io.deephaven.proto.backplane.grpc.UpdateByWindowScale + (*UpdateByEmOptions)(nil), // 24: io.deephaven.proto.backplane.grpc.UpdateByEmOptions + (*UpdateByDeltaOptions)(nil), // 25: io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions + (*UpdateByRequest)(nil), // 26: io.deephaven.proto.backplane.grpc.UpdateByRequest + (*SelectDistinctRequest)(nil), // 27: io.deephaven.proto.backplane.grpc.SelectDistinctRequest + (*DropColumnsRequest)(nil), // 28: io.deephaven.proto.backplane.grpc.DropColumnsRequest + (*UnstructuredFilterTableRequest)(nil), // 29: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest + (*HeadOrTailRequest)(nil), // 30: io.deephaven.proto.backplane.grpc.HeadOrTailRequest + (*HeadOrTailByRequest)(nil), // 31: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest + (*UngroupRequest)(nil), // 32: io.deephaven.proto.backplane.grpc.UngroupRequest + (*MergeTablesRequest)(nil), // 33: io.deephaven.proto.backplane.grpc.MergeTablesRequest + (*SnapshotTableRequest)(nil), // 34: io.deephaven.proto.backplane.grpc.SnapshotTableRequest + (*SnapshotWhenTableRequest)(nil), // 35: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest + (*CrossJoinTablesRequest)(nil), // 36: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest + (*NaturalJoinTablesRequest)(nil), // 37: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest + (*ExactJoinTablesRequest)(nil), // 38: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest + (*LeftJoinTablesRequest)(nil), // 39: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest + (*AsOfJoinTablesRequest)(nil), // 40: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest + (*AjRajTablesRequest)(nil), // 41: io.deephaven.proto.backplane.grpc.AjRajTablesRequest + (*MultiJoinInput)(nil), // 42: io.deephaven.proto.backplane.grpc.MultiJoinInput + (*MultiJoinTablesRequest)(nil), // 43: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest + (*RangeJoinTablesRequest)(nil), // 44: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest + (*ComboAggregateRequest)(nil), // 45: io.deephaven.proto.backplane.grpc.ComboAggregateRequest + (*AggregateAllRequest)(nil), // 46: io.deephaven.proto.backplane.grpc.AggregateAllRequest + (*AggSpec)(nil), // 47: io.deephaven.proto.backplane.grpc.AggSpec + (*AggregateRequest)(nil), // 48: io.deephaven.proto.backplane.grpc.AggregateRequest + (*Aggregation)(nil), // 49: io.deephaven.proto.backplane.grpc.Aggregation + (*SortDescriptor)(nil), // 50: io.deephaven.proto.backplane.grpc.SortDescriptor + (*SortTableRequest)(nil), // 51: io.deephaven.proto.backplane.grpc.SortTableRequest + (*FilterTableRequest)(nil), // 52: io.deephaven.proto.backplane.grpc.FilterTableRequest + (*SeekRowRequest)(nil), // 53: io.deephaven.proto.backplane.grpc.SeekRowRequest + (*SeekRowResponse)(nil), // 54: io.deephaven.proto.backplane.grpc.SeekRowResponse + (*Reference)(nil), // 55: io.deephaven.proto.backplane.grpc.Reference + (*Literal)(nil), // 56: io.deephaven.proto.backplane.grpc.Literal + (*Value)(nil), // 57: io.deephaven.proto.backplane.grpc.Value + (*Condition)(nil), // 58: io.deephaven.proto.backplane.grpc.Condition + (*AndCondition)(nil), // 59: io.deephaven.proto.backplane.grpc.AndCondition + (*OrCondition)(nil), // 60: io.deephaven.proto.backplane.grpc.OrCondition + (*NotCondition)(nil), // 61: io.deephaven.proto.backplane.grpc.NotCondition + (*CompareCondition)(nil), // 62: io.deephaven.proto.backplane.grpc.CompareCondition + (*InCondition)(nil), // 63: io.deephaven.proto.backplane.grpc.InCondition + (*InvokeCondition)(nil), // 64: io.deephaven.proto.backplane.grpc.InvokeCondition + (*IsNullCondition)(nil), // 65: io.deephaven.proto.backplane.grpc.IsNullCondition + (*MatchesCondition)(nil), // 66: io.deephaven.proto.backplane.grpc.MatchesCondition + (*ContainsCondition)(nil), // 67: io.deephaven.proto.backplane.grpc.ContainsCondition + (*SearchCondition)(nil), // 68: io.deephaven.proto.backplane.grpc.SearchCondition + (*FlattenRequest)(nil), // 69: io.deephaven.proto.backplane.grpc.FlattenRequest + (*MetaTableRequest)(nil), // 70: io.deephaven.proto.backplane.grpc.MetaTableRequest + (*RunChartDownsampleRequest)(nil), // 71: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest + (*CreateInputTableRequest)(nil), // 72: io.deephaven.proto.backplane.grpc.CreateInputTableRequest + (*WhereInRequest)(nil), // 73: io.deephaven.proto.backplane.grpc.WhereInRequest + (*ColumnStatisticsRequest)(nil), // 74: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest + (*SliceRequest)(nil), // 75: io.deephaven.proto.backplane.grpc.SliceRequest + (*BatchTableRequest)(nil), // 76: io.deephaven.proto.backplane.grpc.BatchTableRequest + (*UpdateByWindowScale_UpdateByWindowTicks)(nil), // 77: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicks + (*UpdateByWindowScale_UpdateByWindowTime)(nil), // 78: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTime + (*UpdateByRequest_UpdateByOptions)(nil), // 79: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions + (*UpdateByRequest_UpdateByOperation)(nil), // 80: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation + (*UpdateByRequest_UpdateByOperation_UpdateByColumn)(nil), // 81: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec)(nil), // 82: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum)(nil), // 83: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSum + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin)(nil), // 84: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMin + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax)(nil), // 85: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct)(nil), // 86: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProduct + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill)(nil), // 87: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFill + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma)(nil), // 88: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms)(nil), // 89: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin)(nil), // 90: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax)(nil), // 91: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd)(nil), // 92: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta)(nil), // 93: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum)(nil), // 94: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup)(nil), // 95: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg)(nil), // 96: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin)(nil), // 97: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax)(nil), // 98: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct)(nil), // 99: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount)(nil), // 100: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd)(nil), // 101: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg)(nil), // 102: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg + (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula)(nil), // 103: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula + (*ComboAggregateRequest_Aggregate)(nil), // 104: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate + (*AggSpec_AggSpecApproximatePercentile)(nil), // 105: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentile + (*AggSpec_AggSpecCountDistinct)(nil), // 106: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinct + (*AggSpec_AggSpecDistinct)(nil), // 107: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinct + (*AggSpec_AggSpecFormula)(nil), // 108: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormula + (*AggSpec_AggSpecMedian)(nil), // 109: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedian + (*AggSpec_AggSpecPercentile)(nil), // 110: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentile + (*AggSpec_AggSpecSorted)(nil), // 111: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted + (*AggSpec_AggSpecSortedColumn)(nil), // 112: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn + (*AggSpec_AggSpecTDigest)(nil), // 113: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigest + (*AggSpec_AggSpecUnique)(nil), // 114: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique + (*AggSpec_AggSpecNonUniqueSentinel)(nil), // 115: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel + (*AggSpec_AggSpecWeighted)(nil), // 116: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted + (*AggSpec_AggSpecAbsSum)(nil), // 117: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum + (*AggSpec_AggSpecAvg)(nil), // 118: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvg + (*AggSpec_AggSpecFirst)(nil), // 119: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirst + (*AggSpec_AggSpecFreeze)(nil), // 120: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreeze + (*AggSpec_AggSpecGroup)(nil), // 121: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroup + (*AggSpec_AggSpecLast)(nil), // 122: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLast + (*AggSpec_AggSpecMax)(nil), // 123: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMax + (*AggSpec_AggSpecMin)(nil), // 124: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMin + (*AggSpec_AggSpecStd)(nil), // 125: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStd + (*AggSpec_AggSpecSum)(nil), // 126: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSum + (*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 } var file_deephaven_proto_table_proto_depIdxs = []int32{ - 137, // 0: io.deephaven.proto.backplane.grpc.TableReference.ticket:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 3: io.deephaven.proto.backplane.grpc.FetchTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 5: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 137, // 6: io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage.export_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 137, // 7: io.deephaven.proto.backplane.grpc.EmptyTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 137, // 8: io.deephaven.proto.backplane.grpc.TimeTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 137, // 9: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 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 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 - 76, // 12: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.ticks:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicks - 77, // 13: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.time:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTime + 77, // 12: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.ticks:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicks + 78, // 13: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.time:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTime 0, // 14: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_null_value:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior 0, // 15: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_nan_value:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior 0, // 16: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_null_time:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior 0, // 17: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_negative_delta_time:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior 0, // 18: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_zero_delta_time:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior - 21, // 19: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.big_value_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext + 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 - 137, // 21: io.deephaven.proto.backplane.grpc.UpdateByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 78, // 23: io.deephaven.proto.backplane.grpc.UpdateByRequest.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions - 79, // 24: io.deephaven.proto.backplane.grpc.UpdateByRequest.operations:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation - 137, // 25: io.deephaven.proto.backplane.grpc.SelectDistinctRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 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 12, // 26: io.deephaven.proto.backplane.grpc.SelectDistinctRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 137, // 27: io.deephaven.proto.backplane.grpc.DropColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 29: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 31: io.deephaven.proto.backplane.grpc.HeadOrTailRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 33: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 35: io.deephaven.proto.backplane.grpc.UngroupRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 37: io.deephaven.proto.backplane.grpc.MergeTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 39: io.deephaven.proto.backplane.grpc.SnapshotTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 41: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 44: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 47: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 50: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 53: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 56: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 60: io.deephaven.proto.backplane.grpc.AjRajTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 - 137, // 64: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 41, // 65: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest.multi_join_inputs:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinInput - 137, // 66: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 139, // 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 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 - 48, // 71: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.aggregations:type_name -> io.deephaven.proto.backplane.grpc.Aggregation - 137, // 72: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 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 12, // 73: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 103, // 74: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.aggregates:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate - 137, // 75: io.deephaven.proto.backplane.grpc.AggregateAllRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 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 12, // 76: io.deephaven.proto.backplane.grpc.AggregateAllRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 46, // 77: io.deephaven.proto.backplane.grpc.AggregateAllRequest.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec - 116, // 78: io.deephaven.proto.backplane.grpc.AggSpec.abs_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum - 104, // 79: io.deephaven.proto.backplane.grpc.AggSpec.approximate_percentile:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentile - 117, // 80: io.deephaven.proto.backplane.grpc.AggSpec.avg:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvg - 105, // 81: io.deephaven.proto.backplane.grpc.AggSpec.count_distinct:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinct - 106, // 82: io.deephaven.proto.backplane.grpc.AggSpec.distinct:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinct - 118, // 83: io.deephaven.proto.backplane.grpc.AggSpec.first:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirst - 107, // 84: io.deephaven.proto.backplane.grpc.AggSpec.formula:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormula - 119, // 85: io.deephaven.proto.backplane.grpc.AggSpec.freeze:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreeze - 120, // 86: io.deephaven.proto.backplane.grpc.AggSpec.group:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroup - 121, // 87: io.deephaven.proto.backplane.grpc.AggSpec.last:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLast - 122, // 88: io.deephaven.proto.backplane.grpc.AggSpec.max:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMax - 108, // 89: io.deephaven.proto.backplane.grpc.AggSpec.median:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedian - 123, // 90: io.deephaven.proto.backplane.grpc.AggSpec.min:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMin - 109, // 91: io.deephaven.proto.backplane.grpc.AggSpec.percentile:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentile - 110, // 92: io.deephaven.proto.backplane.grpc.AggSpec.sorted_first:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted - 110, // 93: io.deephaven.proto.backplane.grpc.AggSpec.sorted_last:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted - 124, // 94: io.deephaven.proto.backplane.grpc.AggSpec.std:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStd - 125, // 95: io.deephaven.proto.backplane.grpc.AggSpec.sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSum - 112, // 96: io.deephaven.proto.backplane.grpc.AggSpec.t_digest:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigest - 113, // 97: io.deephaven.proto.backplane.grpc.AggSpec.unique:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique - 115, // 98: io.deephaven.proto.backplane.grpc.AggSpec.weighted_avg:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted - 115, // 99: io.deephaven.proto.backplane.grpc.AggSpec.weighted_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted - 126, // 100: io.deephaven.proto.backplane.grpc.AggSpec.var:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVar - 137, // 101: io.deephaven.proto.backplane.grpc.AggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket + 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 + 105, // 79: io.deephaven.proto.backplane.grpc.AggSpec.approximate_percentile:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentile + 118, // 80: io.deephaven.proto.backplane.grpc.AggSpec.avg:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvg + 106, // 81: io.deephaven.proto.backplane.grpc.AggSpec.count_distinct:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinct + 107, // 82: io.deephaven.proto.backplane.grpc.AggSpec.distinct:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinct + 119, // 83: io.deephaven.proto.backplane.grpc.AggSpec.first:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirst + 108, // 84: io.deephaven.proto.backplane.grpc.AggSpec.formula:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormula + 120, // 85: io.deephaven.proto.backplane.grpc.AggSpec.freeze:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreeze + 121, // 86: io.deephaven.proto.backplane.grpc.AggSpec.group:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroup + 122, // 87: io.deephaven.proto.backplane.grpc.AggSpec.last:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLast + 123, // 88: io.deephaven.proto.backplane.grpc.AggSpec.max:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMax + 109, // 89: io.deephaven.proto.backplane.grpc.AggSpec.median:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedian + 124, // 90: io.deephaven.proto.backplane.grpc.AggSpec.min:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMin + 110, // 91: io.deephaven.proto.backplane.grpc.AggSpec.percentile:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentile + 111, // 92: io.deephaven.proto.backplane.grpc.AggSpec.sorted_first:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted + 111, // 93: io.deephaven.proto.backplane.grpc.AggSpec.sorted_last:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted + 125, // 94: io.deephaven.proto.backplane.grpc.AggSpec.std:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStd + 126, // 95: io.deephaven.proto.backplane.grpc.AggSpec.sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSum + 113, // 96: io.deephaven.proto.backplane.grpc.AggSpec.t_digest:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigest + 114, // 97: io.deephaven.proto.backplane.grpc.AggSpec.unique:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique + 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 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 - 48, // 104: io.deephaven.proto.backplane.grpc.AggregateRequest.aggregations:type_name -> io.deephaven.proto.backplane.grpc.Aggregation - 127, // 105: io.deephaven.proto.backplane.grpc.Aggregation.columns:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns - 128, // 106: io.deephaven.proto.backplane.grpc.Aggregation.count:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount - 129, // 107: io.deephaven.proto.backplane.grpc.Aggregation.first_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey - 129, // 108: io.deephaven.proto.backplane.grpc.Aggregation.last_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey - 130, // 109: io.deephaven.proto.backplane.grpc.Aggregation.partition:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition - 10, // 110: io.deephaven.proto.backplane.grpc.SortDescriptor.direction:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection - 137, // 111: io.deephaven.proto.backplane.grpc.SortTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 112: io.deephaven.proto.backplane.grpc.SortTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 49, // 113: io.deephaven.proto.backplane.grpc.SortTableRequest.sorts:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor - 137, // 114: io.deephaven.proto.backplane.grpc.FilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 115: io.deephaven.proto.backplane.grpc.FilterTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 57, // 116: io.deephaven.proto.backplane.grpc.FilterTableRequest.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition - 137, // 117: io.deephaven.proto.backplane.grpc.SeekRowRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 55, // 118: io.deephaven.proto.backplane.grpc.SeekRowRequest.seek_value:type_name -> io.deephaven.proto.backplane.grpc.Literal - 54, // 119: io.deephaven.proto.backplane.grpc.Value.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 55, // 120: io.deephaven.proto.backplane.grpc.Value.literal:type_name -> io.deephaven.proto.backplane.grpc.Literal - 58, // 121: io.deephaven.proto.backplane.grpc.Condition.and:type_name -> io.deephaven.proto.backplane.grpc.AndCondition - 59, // 122: io.deephaven.proto.backplane.grpc.Condition.or:type_name -> io.deephaven.proto.backplane.grpc.OrCondition - 60, // 123: io.deephaven.proto.backplane.grpc.Condition.not:type_name -> io.deephaven.proto.backplane.grpc.NotCondition - 61, // 124: io.deephaven.proto.backplane.grpc.Condition.compare:type_name -> io.deephaven.proto.backplane.grpc.CompareCondition - 62, // 125: io.deephaven.proto.backplane.grpc.Condition.in:type_name -> io.deephaven.proto.backplane.grpc.InCondition - 63, // 126: io.deephaven.proto.backplane.grpc.Condition.invoke:type_name -> io.deephaven.proto.backplane.grpc.InvokeCondition - 64, // 127: io.deephaven.proto.backplane.grpc.Condition.is_null:type_name -> io.deephaven.proto.backplane.grpc.IsNullCondition - 65, // 128: io.deephaven.proto.backplane.grpc.Condition.matches:type_name -> io.deephaven.proto.backplane.grpc.MatchesCondition - 66, // 129: io.deephaven.proto.backplane.grpc.Condition.contains:type_name -> io.deephaven.proto.backplane.grpc.ContainsCondition - 67, // 130: io.deephaven.proto.backplane.grpc.Condition.search:type_name -> io.deephaven.proto.backplane.grpc.SearchCondition - 57, // 131: io.deephaven.proto.backplane.grpc.AndCondition.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition - 57, // 132: io.deephaven.proto.backplane.grpc.OrCondition.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition - 57, // 133: io.deephaven.proto.backplane.grpc.NotCondition.filter:type_name -> io.deephaven.proto.backplane.grpc.Condition - 11, // 134: io.deephaven.proto.backplane.grpc.CompareCondition.operation:type_name -> io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation - 3, // 135: io.deephaven.proto.backplane.grpc.CompareCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 56, // 136: io.deephaven.proto.backplane.grpc.CompareCondition.lhs:type_name -> io.deephaven.proto.backplane.grpc.Value - 56, // 137: io.deephaven.proto.backplane.grpc.CompareCondition.rhs:type_name -> io.deephaven.proto.backplane.grpc.Value - 56, // 138: io.deephaven.proto.backplane.grpc.InCondition.target:type_name -> io.deephaven.proto.backplane.grpc.Value - 56, // 139: io.deephaven.proto.backplane.grpc.InCondition.candidates:type_name -> io.deephaven.proto.backplane.grpc.Value - 3, // 140: io.deephaven.proto.backplane.grpc.InCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 4, // 141: io.deephaven.proto.backplane.grpc.InCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType - 56, // 142: io.deephaven.proto.backplane.grpc.InvokeCondition.target:type_name -> io.deephaven.proto.backplane.grpc.Value - 56, // 143: io.deephaven.proto.backplane.grpc.InvokeCondition.arguments:type_name -> io.deephaven.proto.backplane.grpc.Value - 54, // 144: io.deephaven.proto.backplane.grpc.IsNullCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 54, // 145: io.deephaven.proto.backplane.grpc.MatchesCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 3, // 146: io.deephaven.proto.backplane.grpc.MatchesCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 4, // 147: io.deephaven.proto.backplane.grpc.MatchesCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType - 54, // 148: io.deephaven.proto.backplane.grpc.ContainsCondition.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference - 3, // 149: io.deephaven.proto.backplane.grpc.ContainsCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity - 4, // 150: io.deephaven.proto.backplane.grpc.ContainsCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType - 54, // 151: io.deephaven.proto.backplane.grpc.SearchCondition.optional_references:type_name -> io.deephaven.proto.backplane.grpc.Reference - 137, // 152: io.deephaven.proto.backplane.grpc.FlattenRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 153: io.deephaven.proto.backplane.grpc.FlattenRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 137, // 154: io.deephaven.proto.backplane.grpc.MetaTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 155: io.deephaven.proto.backplane.grpc.MetaTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 137, // 156: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 157: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 131, // 158: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.zoom_range:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange - 137, // 159: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 160: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.source_table_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 132, // 161: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.kind:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind - 137, // 162: io.deephaven.proto.backplane.grpc.WhereInRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 163: io.deephaven.proto.backplane.grpc.WhereInRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 12, // 164: io.deephaven.proto.backplane.grpc.WhereInRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 137, // 165: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 166: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 137, // 167: io.deephaven.proto.backplane.grpc.SliceRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket - 12, // 168: io.deephaven.proto.backplane.grpc.SliceRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference - 136, // 169: io.deephaven.proto.backplane.grpc.BatchTableRequest.ops:type_name -> io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation - 21, // 170: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions.math_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext - 80, // 171: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.column:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn - 81, // 172: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.spec:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec - 82, // 173: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.sum:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSum - 83, // 174: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMin - 84, // 175: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMax - 85, // 176: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.product:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProduct - 86, // 177: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.fill:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFill - 87, // 178: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.ema:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma - 93, // 179: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_sum:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum - 94, // 180: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_group:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup - 95, // 181: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_avg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg - 96, // 182: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin - 97, // 183: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax - 98, // 184: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_product:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct - 92, // 185: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.delta:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta - 88, // 186: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.ems:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms - 89, // 187: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_min:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin - 90, // 188: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_max:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax - 91, // 189: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.em_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd - 99, // 190: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_count:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount - 100, // 191: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd - 101, // 192: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_wavg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg - 102, // 193: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_formula:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula - 23, // 194: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 22, // 195: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 196: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 22, // 197: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 198: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 22, // 199: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 200: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 22, // 201: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 23, // 202: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions - 22, // 203: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 24, // 204: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions - 22, // 205: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 206: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 207: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 208: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 209: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 210: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 211: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 212: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 213: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 214: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 215: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 216: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 217: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 218: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 219: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 220: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 221: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 222: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 223: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 22, // 224: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale - 9, // 225: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate.type:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType - 111, // 226: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted.columns:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn - 114, // 227: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique.non_unique_sentinel:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel - 2, // 228: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel.null_value:type_name -> io.deephaven.proto.backplane.grpc.NullValue - 46, // 229: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec - 133, // 230: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_append_only:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly - 134, // 231: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_key_backed:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked - 135, // 232: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.blink:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink - 18, // 233: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.empty_table:type_name -> io.deephaven.proto.backplane.grpc.EmptyTableRequest - 19, // 234: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.time_table:type_name -> io.deephaven.proto.backplane.grpc.TimeTableRequest - 27, // 235: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.drop_columns:type_name -> io.deephaven.proto.backplane.grpc.DropColumnsRequest - 20, // 236: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 237: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.lazy_update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 238: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 239: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 240: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 26, // 241: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select_distinct:type_name -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest - 51, // 242: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.filter:type_name -> io.deephaven.proto.backplane.grpc.FilterTableRequest - 28, // 243: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.unstructured_filter:type_name -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest - 50, // 244: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.sort:type_name -> io.deephaven.proto.backplane.grpc.SortTableRequest - 29, // 245: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 29, // 246: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 30, // 247: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 30, // 248: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 31, // 249: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.ungroup:type_name -> io.deephaven.proto.backplane.grpc.UngroupRequest - 32, // 250: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.merge:type_name -> io.deephaven.proto.backplane.grpc.MergeTablesRequest - 44, // 251: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.combo_aggregate:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest - 68, // 252: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.flatten:type_name -> io.deephaven.proto.backplane.grpc.FlattenRequest - 70, // 253: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.run_chart_downsample:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest - 35, // 254: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.cross_join:type_name -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest - 36, // 255: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.natural_join:type_name -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest - 37, // 256: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.exact_join:type_name -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest - 38, // 257: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.left_join:type_name -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest - 39, // 258: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.as_of_join:type_name -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest - 14, // 259: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.fetch_table:type_name -> io.deephaven.proto.backplane.grpc.FetchTableRequest - 15, // 260: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.apply_preview_columns:type_name -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest - 71, // 261: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.create_input_table:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest - 25, // 262: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_by:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest - 72, // 263: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.where_in:type_name -> io.deephaven.proto.backplane.grpc.WhereInRequest - 45, // 264: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate_all:type_name -> io.deephaven.proto.backplane.grpc.AggregateAllRequest - 47, // 265: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate:type_name -> io.deephaven.proto.backplane.grpc.AggregateRequest - 33, // 266: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot:type_name -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest - 34, // 267: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot_when:type_name -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest - 69, // 268: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.meta_table:type_name -> io.deephaven.proto.backplane.grpc.MetaTableRequest - 43, // 269: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.range_join:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest - 40, // 270: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 40, // 271: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.raj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 73, // 272: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.column_statistics:type_name -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest - 42, // 273: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.multi_join:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest - 74, // 274: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.slice:type_name -> io.deephaven.proto.backplane.grpc.SliceRequest - 137, // 275: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:input_type -> io.deephaven.proto.backplane.grpc.Ticket - 14, // 276: io.deephaven.proto.backplane.grpc.TableService.FetchTable:input_type -> io.deephaven.proto.backplane.grpc.FetchTableRequest - 15, // 277: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:input_type -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest - 18, // 278: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:input_type -> io.deephaven.proto.backplane.grpc.EmptyTableRequest - 19, // 279: io.deephaven.proto.backplane.grpc.TableService.TimeTable:input_type -> io.deephaven.proto.backplane.grpc.TimeTableRequest - 27, // 280: io.deephaven.proto.backplane.grpc.TableService.DropColumns:input_type -> io.deephaven.proto.backplane.grpc.DropColumnsRequest - 20, // 281: io.deephaven.proto.backplane.grpc.TableService.Update:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 282: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 283: io.deephaven.proto.backplane.grpc.TableService.View:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 284: io.deephaven.proto.backplane.grpc.TableService.UpdateView:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 20, // 285: io.deephaven.proto.backplane.grpc.TableService.Select:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest - 25, // 286: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:input_type -> io.deephaven.proto.backplane.grpc.UpdateByRequest - 26, // 287: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:input_type -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest - 51, // 288: io.deephaven.proto.backplane.grpc.TableService.Filter:input_type -> io.deephaven.proto.backplane.grpc.FilterTableRequest - 28, // 289: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:input_type -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest - 50, // 290: io.deephaven.proto.backplane.grpc.TableService.Sort:input_type -> io.deephaven.proto.backplane.grpc.SortTableRequest - 29, // 291: io.deephaven.proto.backplane.grpc.TableService.Head:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 29, // 292: io.deephaven.proto.backplane.grpc.TableService.Tail:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest - 30, // 293: io.deephaven.proto.backplane.grpc.TableService.HeadBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 30, // 294: io.deephaven.proto.backplane.grpc.TableService.TailBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest - 31, // 295: io.deephaven.proto.backplane.grpc.TableService.Ungroup:input_type -> io.deephaven.proto.backplane.grpc.UngroupRequest - 32, // 296: io.deephaven.proto.backplane.grpc.TableService.MergeTables:input_type -> io.deephaven.proto.backplane.grpc.MergeTablesRequest - 35, // 297: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:input_type -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest - 36, // 298: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:input_type -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest - 37, // 299: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:input_type -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest - 38, // 300: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:input_type -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest - 39, // 301: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:input_type -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest - 40, // 302: io.deephaven.proto.backplane.grpc.TableService.AjTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 40, // 303: io.deephaven.proto.backplane.grpc.TableService.RajTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest - 42, // 304: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:input_type -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest - 43, // 305: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:input_type -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest - 44, // 306: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:input_type -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest - 45, // 307: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:input_type -> io.deephaven.proto.backplane.grpc.AggregateAllRequest - 47, // 308: io.deephaven.proto.backplane.grpc.TableService.Aggregate:input_type -> io.deephaven.proto.backplane.grpc.AggregateRequest - 33, // 309: io.deephaven.proto.backplane.grpc.TableService.Snapshot:input_type -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest - 34, // 310: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:input_type -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest - 68, // 311: io.deephaven.proto.backplane.grpc.TableService.Flatten:input_type -> io.deephaven.proto.backplane.grpc.FlattenRequest - 70, // 312: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:input_type -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest - 71, // 313: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:input_type -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest - 72, // 314: io.deephaven.proto.backplane.grpc.TableService.WhereIn:input_type -> io.deephaven.proto.backplane.grpc.WhereInRequest - 75, // 315: io.deephaven.proto.backplane.grpc.TableService.Batch:input_type -> io.deephaven.proto.backplane.grpc.BatchTableRequest - 16, // 316: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:input_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest - 52, // 317: io.deephaven.proto.backplane.grpc.TableService.SeekRow:input_type -> io.deephaven.proto.backplane.grpc.SeekRowRequest - 69, // 318: io.deephaven.proto.backplane.grpc.TableService.MetaTable:input_type -> io.deephaven.proto.backplane.grpc.MetaTableRequest - 73, // 319: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:input_type -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest - 74, // 320: io.deephaven.proto.backplane.grpc.TableService.Slice:input_type -> io.deephaven.proto.backplane.grpc.SliceRequest - 13, // 321: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 322: io.deephaven.proto.backplane.grpc.TableService.FetchTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 323: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 324: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 325: io.deephaven.proto.backplane.grpc.TableService.TimeTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 326: io.deephaven.proto.backplane.grpc.TableService.DropColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 327: io.deephaven.proto.backplane.grpc.TableService.Update:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 328: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 329: io.deephaven.proto.backplane.grpc.TableService.View:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 330: io.deephaven.proto.backplane.grpc.TableService.UpdateView:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 331: io.deephaven.proto.backplane.grpc.TableService.Select:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 332: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 333: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 334: io.deephaven.proto.backplane.grpc.TableService.Filter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 335: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 336: io.deephaven.proto.backplane.grpc.TableService.Sort:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 337: io.deephaven.proto.backplane.grpc.TableService.Head:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 338: io.deephaven.proto.backplane.grpc.TableService.Tail:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 339: io.deephaven.proto.backplane.grpc.TableService.HeadBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 340: io.deephaven.proto.backplane.grpc.TableService.TailBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 341: io.deephaven.proto.backplane.grpc.TableService.Ungroup:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 342: io.deephaven.proto.backplane.grpc.TableService.MergeTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 343: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 344: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 345: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 346: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 347: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 348: io.deephaven.proto.backplane.grpc.TableService.AjTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 349: io.deephaven.proto.backplane.grpc.TableService.RajTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 350: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 351: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 352: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 353: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 354: io.deephaven.proto.backplane.grpc.TableService.Aggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 355: io.deephaven.proto.backplane.grpc.TableService.Snapshot:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 356: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 357: io.deephaven.proto.backplane.grpc.TableService.Flatten:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 358: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 359: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 360: io.deephaven.proto.backplane.grpc.TableService.WhereIn:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 361: io.deephaven.proto.backplane.grpc.TableService.Batch:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 17, // 362: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage - 53, // 363: io.deephaven.proto.backplane.grpc.TableService.SeekRow:output_type -> io.deephaven.proto.backplane.grpc.SeekRowResponse - 13, // 364: io.deephaven.proto.backplane.grpc.TableService.MetaTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 365: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 13, // 366: io.deephaven.proto.backplane.grpc.TableService.Slice:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse - 321, // [321:367] is the sub-list for method output_type - 275, // [275:321] is the sub-list for method input_type - 275, // [275:275] is the sub-list for extension type_name - 275, // [275:275] is the sub-list for extension extendee - 0, // [0:275] is the sub-list for field type_name + 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 } func init() { file_deephaven_proto_table_proto_init() } @@ -13147,7 +13298,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MathContext); i { + switch v := v.(*Selectable); i { case 0: return &v.state case 1: @@ -13159,7 +13310,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByWindowScale); i { + switch v := v.(*MathContext); i { case 0: return &v.state case 1: @@ -13171,7 +13322,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByEmOptions); i { + switch v := v.(*UpdateByWindowScale); i { case 0: return &v.state case 1: @@ -13183,7 +13334,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByDeltaOptions); i { + switch v := v.(*UpdateByEmOptions); i { case 0: return &v.state case 1: @@ -13195,7 +13346,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest); i { + switch v := v.(*UpdateByDeltaOptions); i { case 0: return &v.state case 1: @@ -13207,7 +13358,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SelectDistinctRequest); i { + switch v := v.(*UpdateByRequest); i { case 0: return &v.state case 1: @@ -13219,7 +13370,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DropColumnsRequest); i { + switch v := v.(*SelectDistinctRequest); i { case 0: return &v.state case 1: @@ -13231,7 +13382,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UnstructuredFilterTableRequest); i { + switch v := v.(*DropColumnsRequest); i { case 0: return &v.state case 1: @@ -13243,7 +13394,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HeadOrTailRequest); i { + switch v := v.(*UnstructuredFilterTableRequest); i { case 0: return &v.state case 1: @@ -13255,7 +13406,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*HeadOrTailByRequest); i { + switch v := v.(*HeadOrTailRequest); i { case 0: return &v.state case 1: @@ -13267,7 +13418,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UngroupRequest); i { + switch v := v.(*HeadOrTailByRequest); i { case 0: return &v.state case 1: @@ -13279,7 +13430,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MergeTablesRequest); i { + switch v := v.(*UngroupRequest); i { case 0: return &v.state case 1: @@ -13291,7 +13442,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SnapshotTableRequest); i { + switch v := v.(*MergeTablesRequest); i { case 0: return &v.state case 1: @@ -13303,7 +13454,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SnapshotWhenTableRequest); i { + switch v := v.(*SnapshotTableRequest); i { case 0: return &v.state case 1: @@ -13315,7 +13466,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CrossJoinTablesRequest); i { + switch v := v.(*SnapshotWhenTableRequest); i { case 0: return &v.state case 1: @@ -13327,7 +13478,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NaturalJoinTablesRequest); i { + switch v := v.(*CrossJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13339,7 +13490,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExactJoinTablesRequest); i { + switch v := v.(*NaturalJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13351,7 +13502,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LeftJoinTablesRequest); i { + switch v := v.(*ExactJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13363,7 +13514,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AsOfJoinTablesRequest); i { + switch v := v.(*LeftJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13375,7 +13526,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AjRajTablesRequest); i { + switch v := v.(*AsOfJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13387,7 +13538,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MultiJoinInput); i { + switch v := v.(*AjRajTablesRequest); i { case 0: return &v.state case 1: @@ -13399,7 +13550,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MultiJoinTablesRequest); i { + switch v := v.(*MultiJoinInput); i { case 0: return &v.state case 1: @@ -13411,7 +13562,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RangeJoinTablesRequest); i { + switch v := v.(*MultiJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13423,7 +13574,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ComboAggregateRequest); i { + switch v := v.(*RangeJoinTablesRequest); i { case 0: return &v.state case 1: @@ -13435,7 +13586,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggregateAllRequest); i { + switch v := v.(*ComboAggregateRequest); i { case 0: return &v.state case 1: @@ -13447,7 +13598,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec); i { + switch v := v.(*AggregateAllRequest); i { case 0: return &v.state case 1: @@ -13459,7 +13610,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggregateRequest); i { + switch v := v.(*AggSpec); i { case 0: return &v.state case 1: @@ -13471,7 +13622,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation); i { + switch v := v.(*AggregateRequest); i { case 0: return &v.state case 1: @@ -13483,7 +13634,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SortDescriptor); i { + switch v := v.(*Aggregation); i { case 0: return &v.state case 1: @@ -13495,7 +13646,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SortTableRequest); i { + switch v := v.(*SortDescriptor); i { case 0: return &v.state case 1: @@ -13507,7 +13658,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FilterTableRequest); i { + switch v := v.(*SortTableRequest); i { case 0: return &v.state case 1: @@ -13519,7 +13670,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SeekRowRequest); i { + switch v := v.(*FilterTableRequest); i { case 0: return &v.state case 1: @@ -13531,7 +13682,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SeekRowResponse); i { + switch v := v.(*SeekRowRequest); i { case 0: return &v.state case 1: @@ -13543,7 +13694,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Reference); i { + switch v := v.(*SeekRowResponse); i { case 0: return &v.state case 1: @@ -13555,7 +13706,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Literal); i { + switch v := v.(*Reference); i { case 0: return &v.state case 1: @@ -13567,7 +13718,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Value); i { + switch v := v.(*Literal); i { case 0: return &v.state case 1: @@ -13579,7 +13730,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Condition); i { + switch v := v.(*Value); i { case 0: return &v.state case 1: @@ -13591,7 +13742,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AndCondition); i { + switch v := v.(*Condition); i { case 0: return &v.state case 1: @@ -13603,7 +13754,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*OrCondition); i { + switch v := v.(*AndCondition); i { case 0: return &v.state case 1: @@ -13615,7 +13766,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NotCondition); i { + switch v := v.(*OrCondition); i { case 0: return &v.state case 1: @@ -13627,7 +13778,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CompareCondition); i { + switch v := v.(*NotCondition); i { case 0: return &v.state case 1: @@ -13639,7 +13790,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InCondition); i { + switch v := v.(*CompareCondition); i { case 0: return &v.state case 1: @@ -13651,7 +13802,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*InvokeCondition); i { + switch v := v.(*InCondition); i { case 0: return &v.state case 1: @@ -13663,7 +13814,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*IsNullCondition); i { + switch v := v.(*InvokeCondition); i { case 0: return &v.state case 1: @@ -13675,7 +13826,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MatchesCondition); i { + switch v := v.(*IsNullCondition); i { case 0: return &v.state case 1: @@ -13687,7 +13838,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ContainsCondition); i { + switch v := v.(*MatchesCondition); i { case 0: return &v.state case 1: @@ -13699,7 +13850,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SearchCondition); i { + switch v := v.(*ContainsCondition); i { case 0: return &v.state case 1: @@ -13711,7 +13862,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*FlattenRequest); i { + switch v := v.(*SearchCondition); i { case 0: return &v.state case 1: @@ -13723,7 +13874,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*MetaTableRequest); i { + switch v := v.(*FlattenRequest); i { case 0: return &v.state case 1: @@ -13735,7 +13886,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunChartDownsampleRequest); i { + switch v := v.(*MetaTableRequest); i { case 0: return &v.state case 1: @@ -13747,7 +13898,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest); i { + switch v := v.(*RunChartDownsampleRequest); i { case 0: return &v.state case 1: @@ -13759,7 +13910,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*WhereInRequest); i { + switch v := v.(*CreateInputTableRequest); i { case 0: return &v.state case 1: @@ -13771,7 +13922,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ColumnStatisticsRequest); i { + switch v := v.(*WhereInRequest); i { case 0: return &v.state case 1: @@ -13783,7 +13934,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SliceRequest); i { + switch v := v.(*ColumnStatisticsRequest); i { case 0: return &v.state case 1: @@ -13795,7 +13946,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BatchTableRequest); i { + switch v := v.(*SliceRequest); i { case 0: return &v.state case 1: @@ -13807,7 +13958,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByWindowScale_UpdateByWindowTicks); i { + switch v := v.(*BatchTableRequest); i { case 0: return &v.state case 1: @@ -13819,7 +13970,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByWindowScale_UpdateByWindowTime); i { + switch v := v.(*UpdateByWindowScale_UpdateByWindowTicks); i { case 0: return &v.state case 1: @@ -13831,7 +13982,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOptions); i { + switch v := v.(*UpdateByWindowScale_UpdateByWindowTime); i { case 0: return &v.state case 1: @@ -13843,7 +13994,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation); i { + switch v := v.(*UpdateByRequest_UpdateByOptions); i { case 0: return &v.state case 1: @@ -13855,7 +14006,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn); i { + switch v := v.(*UpdateByRequest_UpdateByOperation); i { case 0: return &v.state case 1: @@ -13867,7 +14018,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn); i { case 0: return &v.state case 1: @@ -13879,7 +14030,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec); i { case 0: return &v.state case 1: @@ -13891,7 +14042,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum); i { case 0: return &v.state case 1: @@ -13903,7 +14054,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMin); i { case 0: return &v.state case 1: @@ -13915,7 +14066,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeMax); i { case 0: return &v.state case 1: @@ -13927,7 +14078,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeProduct); i { case 0: return &v.state case 1: @@ -13939,7 +14090,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByFill); i { case 0: return &v.state case 1: @@ -13951,7 +14102,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEma); i { case 0: return &v.state case 1: @@ -13963,7 +14114,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEms); i { case 0: return &v.state case 1: @@ -13975,7 +14126,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMin); i { case 0: return &v.state case 1: @@ -13987,7 +14138,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmMax); i { case 0: return &v.state case 1: @@ -13999,7 +14150,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByEmStd); i { case 0: return &v.state case 1: @@ -14011,7 +14162,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByDelta); i { case 0: return &v.state case 1: @@ -14023,7 +14174,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingSum); i { case 0: return &v.state case 1: @@ -14035,7 +14186,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingGroup); i { case 0: return &v.state case 1: @@ -14047,7 +14198,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingAvg); i { case 0: return &v.state case 1: @@ -14059,7 +14210,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMin); i { case 0: return &v.state case 1: @@ -14071,7 +14222,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingMax); i { case 0: return &v.state case 1: @@ -14083,7 +14234,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingProduct); i { case 0: return &v.state case 1: @@ -14095,7 +14246,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCount); i { case 0: return &v.state case 1: @@ -14107,7 +14258,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingStd); i { case 0: return &v.state case 1: @@ -14119,7 +14270,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingWAvg); i { case 0: return &v.state case 1: @@ -14131,7 +14282,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ComboAggregateRequest_Aggregate); i { + switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula); i { case 0: return &v.state case 1: @@ -14143,7 +14294,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecApproximatePercentile); i { + switch v := v.(*ComboAggregateRequest_Aggregate); i { case 0: return &v.state case 1: @@ -14155,7 +14306,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecCountDistinct); i { + switch v := v.(*AggSpec_AggSpecApproximatePercentile); i { case 0: return &v.state case 1: @@ -14167,7 +14318,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecDistinct); i { + switch v := v.(*AggSpec_AggSpecCountDistinct); i { case 0: return &v.state case 1: @@ -14179,7 +14330,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecFormula); i { + switch v := v.(*AggSpec_AggSpecDistinct); i { case 0: return &v.state case 1: @@ -14191,7 +14342,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecMedian); i { + switch v := v.(*AggSpec_AggSpecFormula); i { case 0: return &v.state case 1: @@ -14203,7 +14354,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecPercentile); i { + switch v := v.(*AggSpec_AggSpecMedian); i { case 0: return &v.state case 1: @@ -14215,7 +14366,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecSorted); i { + switch v := v.(*AggSpec_AggSpecPercentile); i { case 0: return &v.state case 1: @@ -14227,7 +14378,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecSortedColumn); i { + switch v := v.(*AggSpec_AggSpecSorted); i { case 0: return &v.state case 1: @@ -14239,7 +14390,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecTDigest); i { + switch v := v.(*AggSpec_AggSpecSortedColumn); i { case 0: return &v.state case 1: @@ -14251,7 +14402,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecUnique); i { + switch v := v.(*AggSpec_AggSpecTDigest); i { case 0: return &v.state case 1: @@ -14263,7 +14414,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecNonUniqueSentinel); i { + switch v := v.(*AggSpec_AggSpecUnique); i { case 0: return &v.state case 1: @@ -14275,7 +14426,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecWeighted); i { + switch v := v.(*AggSpec_AggSpecNonUniqueSentinel); i { case 0: return &v.state case 1: @@ -14287,7 +14438,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecAbsSum); i { + switch v := v.(*AggSpec_AggSpecWeighted); i { case 0: return &v.state case 1: @@ -14299,7 +14450,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecAvg); i { + switch v := v.(*AggSpec_AggSpecAbsSum); i { case 0: return &v.state case 1: @@ -14311,7 +14462,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecFirst); i { + switch v := v.(*AggSpec_AggSpecAvg); i { case 0: return &v.state case 1: @@ -14323,7 +14474,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecFreeze); i { + switch v := v.(*AggSpec_AggSpecFirst); i { case 0: return &v.state case 1: @@ -14335,7 +14486,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecGroup); i { + switch v := v.(*AggSpec_AggSpecFreeze); i { case 0: return &v.state case 1: @@ -14347,7 +14498,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecLast); i { + switch v := v.(*AggSpec_AggSpecGroup); i { case 0: return &v.state case 1: @@ -14359,7 +14510,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecMax); i { + switch v := v.(*AggSpec_AggSpecLast); i { case 0: return &v.state case 1: @@ -14371,7 +14522,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecMin); i { + switch v := v.(*AggSpec_AggSpecMax); i { case 0: return &v.state case 1: @@ -14383,7 +14534,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecStd); i { + switch v := v.(*AggSpec_AggSpecMin); i { case 0: return &v.state case 1: @@ -14395,7 +14546,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecSum); i { + switch v := v.(*AggSpec_AggSpecStd); i { case 0: return &v.state case 1: @@ -14407,7 +14558,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AggSpec_AggSpecVar); i { + switch v := v.(*AggSpec_AggSpecSum); i { case 0: return &v.state case 1: @@ -14419,7 +14570,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation_AggregationColumns); i { + switch v := v.(*AggSpec_AggSpecVar); i { case 0: return &v.state case 1: @@ -14431,7 +14582,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation_AggregationCount); i { + switch v := v.(*Aggregation_AggregationColumns); i { case 0: return &v.state case 1: @@ -14443,7 +14594,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Aggregation_AggregationRowKey); i { + switch v := v.(*Aggregation_AggregationCount); i { case 0: return &v.state case 1: @@ -14455,7 +14606,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[118].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: @@ -14467,7 +14618,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RunChartDownsampleRequest_ZoomRange); i { + switch v := v.(*Aggregation_AggregationPartition); i { case 0: return &v.state case 1: @@ -14479,7 +14630,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind); i { + switch v := v.(*Aggregation_AggregationFormula); i { case 0: return &v.state case 1: @@ -14491,7 +14642,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly); i { + switch v := v.(*RunChartDownsampleRequest_ZoomRange); i { case 0: return &v.state case 1: @@ -14503,7 +14654,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked); i { + switch v := v.(*CreateInputTableRequest_InputTableKind); i { case 0: return &v.state case 1: @@ -14515,7 +14666,7 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CreateInputTableRequest_InputTableKind_Blink); i { + switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly); i { case 0: return &v.state case 1: @@ -14527,6 +14678,30 @@ func file_deephaven_proto_table_proto_init() { } } file_deephaven_proto_table_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_deephaven_proto_table_proto_msgTypes[125].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_proto_table_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BatchTableRequest_Operation); i { case 0: return &v.state @@ -14549,11 +14724,14 @@ func file_deephaven_proto_table_proto_init() { (*TimeTableRequest_PeriodNanos)(nil), (*TimeTableRequest_PeriodString)(nil), } - file_deephaven_proto_table_proto_msgTypes[10].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[9].OneofWrappers = []interface{}{ + (*Selectable_Raw)(nil), + } + file_deephaven_proto_table_proto_msgTypes[11].OneofWrappers = []interface{}{ (*UpdateByWindowScale_Ticks)(nil), (*UpdateByWindowScale_Time)(nil), } - file_deephaven_proto_table_proto_msgTypes[34].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[35].OneofWrappers = []interface{}{ (*AggSpec_AbsSum)(nil), (*AggSpec_ApproximatePercentile)(nil), (*AggSpec_Avg)(nil), @@ -14578,25 +14756,26 @@ func file_deephaven_proto_table_proto_init() { (*AggSpec_WeightedSum)(nil), (*AggSpec_Var)(nil), } - file_deephaven_proto_table_proto_msgTypes[36].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[37].OneofWrappers = []interface{}{ (*Aggregation_Columns)(nil), (*Aggregation_Count)(nil), (*Aggregation_FirstRowKey)(nil), (*Aggregation_LastRowKey)(nil), (*Aggregation_Partition)(nil), + (*Aggregation_Formula)(nil), } - file_deephaven_proto_table_proto_msgTypes[43].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[44].OneofWrappers = []interface{}{ (*Literal_StringValue)(nil), (*Literal_DoubleValue)(nil), (*Literal_BoolValue)(nil), (*Literal_LongValue)(nil), (*Literal_NanoTimeValue)(nil), } - file_deephaven_proto_table_proto_msgTypes[44].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[45].OneofWrappers = []interface{}{ (*Value_Reference)(nil), (*Value_Literal)(nil), } - file_deephaven_proto_table_proto_msgTypes[45].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[46].OneofWrappers = []interface{}{ (*Condition_And)(nil), (*Condition_Or)(nil), (*Condition_Not)(nil), @@ -14608,20 +14787,20 @@ func file_deephaven_proto_table_proto_init() { (*Condition_Contains)(nil), (*Condition_Search)(nil), } - file_deephaven_proto_table_proto_msgTypes[59].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[60].OneofWrappers = []interface{}{ (*CreateInputTableRequest_SourceTableId)(nil), (*CreateInputTableRequest_Schema)(nil), } - file_deephaven_proto_table_proto_msgTypes[61].OneofWrappers = []interface{}{} - file_deephaven_proto_table_proto_msgTypes[65].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[62].OneofWrappers = []interface{}{} + file_deephaven_proto_table_proto_msgTypes[66].OneofWrappers = []interface{}{ (*UpdateByWindowScale_UpdateByWindowTime_Nanos)(nil), (*UpdateByWindowScale_UpdateByWindowTime_DurationString)(nil), } - file_deephaven_proto_table_proto_msgTypes[66].OneofWrappers = []interface{}{} - file_deephaven_proto_table_proto_msgTypes[67].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[67].OneofWrappers = []interface{}{} + file_deephaven_proto_table_proto_msgTypes[68].OneofWrappers = []interface{}{ (*UpdateByRequest_UpdateByOperation_Column)(nil), } - file_deephaven_proto_table_proto_msgTypes[69].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[70].OneofWrappers = []interface{}{ (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Sum)(nil), (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Min)(nil), (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Max)(nil), @@ -14644,9 +14823,9 @@ func file_deephaven_proto_table_proto_init() { (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingWavg)(nil), (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingFormula)(nil), } - file_deephaven_proto_table_proto_msgTypes[92].OneofWrappers = []interface{}{} - file_deephaven_proto_table_proto_msgTypes[100].OneofWrappers = []interface{}{} - file_deephaven_proto_table_proto_msgTypes[102].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[93].OneofWrappers = []interface{}{} + file_deephaven_proto_table_proto_msgTypes[101].OneofWrappers = []interface{}{} + file_deephaven_proto_table_proto_msgTypes[103].OneofWrappers = []interface{}{ (*AggSpec_AggSpecNonUniqueSentinel_NullValue)(nil), (*AggSpec_AggSpecNonUniqueSentinel_StringValue)(nil), (*AggSpec_AggSpecNonUniqueSentinel_IntValue)(nil), @@ -14658,13 +14837,13 @@ func file_deephaven_proto_table_proto_init() { (*AggSpec_AggSpecNonUniqueSentinel_ShortValue)(nil), (*AggSpec_AggSpecNonUniqueSentinel_CharValue)(nil), } - file_deephaven_proto_table_proto_msgTypes[119].OneofWrappers = []interface{}{} - file_deephaven_proto_table_proto_msgTypes[120].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[121].OneofWrappers = []interface{}{} + file_deephaven_proto_table_proto_msgTypes[122].OneofWrappers = []interface{}{ (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly_)(nil), (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked_)(nil), (*CreateInputTableRequest_InputTableKind_Blink_)(nil), } - file_deephaven_proto_table_proto_msgTypes[124].OneofWrappers = []interface{}{ + file_deephaven_proto_table_proto_msgTypes[126].OneofWrappers = []interface{}{ (*BatchTableRequest_Operation_EmptyTable)(nil), (*BatchTableRequest_Operation_TimeTable)(nil), (*BatchTableRequest_Operation_DropColumns)(nil), @@ -14714,7 +14893,7 @@ func file_deephaven_proto_table_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_deephaven_proto_table_proto_rawDesc, NumEnums: 12, - NumMessages: 125, + NumMessages: 127, 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 15b8a2adc9a..05238bf3bcd 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 @@ -11,6 +11,7 @@ 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; @@ -182,5 +183,14 @@ public void visit(Partition partition) { out.add(Partition.of("Output", false)); out.add(Partition.of("Output", true)); } + + @Override + public void visit(Formula formula) { + out.add(Formula.of("Formula", "3.0 + 1")); + out.add(Formula.of("Formula", "sum(S)")); + out.add(Formula.of("Formula", "sum(S + I)")); + out.add(Formula.of("Formula", "sum(S + I + D)")); + out.add(Formula.of("Formula", "sum(S + I + D + L)")); + } } } 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 e44adc6f33a..38e3679b561 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 @@ -9,15 +9,18 @@ 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.Pair; import io.deephaven.api.agg.Partition; 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.AggregationFormula; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey; import io.deephaven.proto.backplane.grpc.Aggregation.Builder; +import io.deephaven.proto.backplane.grpc.Selectable; import java.util.Collection; import java.util.List; @@ -95,4 +98,11 @@ public void visit(Partition partition) { .setColumnName(partition.column().name()) .setIncludeGroupByColumns(partition.includeGroupByColumns()))); } + + @Override + public void visit(Formula formula) { + final Selectable selectable = Selectable.newBuilder().setRaw(formula.formulaString()).build(); + out = singletonList(of(Builder::setFormula, AggregationFormula.newBuilder() + .setSelectable(selectable))); + } } diff --git a/proto/proto-backplane-grpc/src/main/proto/deephaven/proto/table.proto b/proto/proto-backplane-grpc/src/main/proto/deephaven/proto/table.proto index 7fc77ff238d..6e4b90300f0 100644 --- a/proto/proto-backplane-grpc/src/main/proto/deephaven/proto/table.proto +++ b/proto/proto-backplane-grpc/src/main/proto/deephaven/proto/table.proto @@ -348,6 +348,17 @@ message SelectOrUpdateRequest { repeated string column_specs = 3; } +message Selectable { + // message ColumnExpression { + // string column_name = 1; + // Expression expression = 2; + // } + oneof type { + string raw = 1; + // ColumnExpression column_expression = 2; + } +} + message MathContext { enum RoundingMode { ROUNDING_MODE_NOT_SPECIFIED = 0; @@ -1049,12 +1060,17 @@ message Aggregation { bool include_group_by_columns = 2; } + message AggregationFormula { + Selectable selectable = 1; + } + oneof type { AggregationColumns columns = 1; AggregationCount count = 2; AggregationRowKey first_row_key = 3; AggregationRowKey last_row_key = 4; AggregationPartition partition = 5; + AggregationFormula formula = 6; } } diff --git a/py/client/pydeephaven/agg.py b/py/client/pydeephaven/agg.py index 5d95004231b..e4dcb710bc0 100644 --- a/py/client/pydeephaven/agg.py +++ b/py/client/pydeephaven/agg.py @@ -5,16 +5,18 @@ from abc import ABC, abstractmethod from dataclasses import dataclass -from typing import List, Union, Any +from typing import List, Union, Any, Optional import numpy as np from pydeephaven._utils import to_list from pydeephaven.proto import table_pb2 +_GrpcSelectable = table_pb2.Selectable _GrpcAggregation = table_pb2.Aggregation _GrpcAggregationColumns = _GrpcAggregation.AggregationColumns _GrpcAggregationCount = _GrpcAggregation.AggregationCount +_GrpcAggregationFormula = _GrpcAggregation.AggregationFormula _GrpcAggregationPartition = _GrpcAggregation.AggregationPartition _GrpcAggSpec = table_pb2.AggSpec _GrpcNullValue = table_pb2.NullValue @@ -49,6 +51,13 @@ def make_grpc_message(self) -> _GrpcAggregation: agg_count = _GrpcAggregationCount(column_name=self.col) return _GrpcAggregation(count=agg_count) +@dataclass +class _AggregationFormula(Aggregation): + selectable: _GrpcSelectable + + def make_grpc_message(self) -> _GrpcAggregation: + agg_formula = _GrpcAggregationFormula(selectable=self.selectable) + return _GrpcAggregation(formula=agg_formula) @dataclass class _AggregationPartition(Aggregation): @@ -64,7 +73,7 @@ def sum_(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Sum aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -78,7 +87,7 @@ def abs_sum(cols: Union[str, List[str]] = None) -> Aggregation: """Creates an Absolute-sum aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -92,7 +101,7 @@ def group(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Group aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -106,7 +115,7 @@ def avg(cols: Union[str, List[str]] = None) -> Aggregation: """Creates an Average aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -146,7 +155,7 @@ def count_distinct(cols: Union[str, List[str]] = None, count_nulls: bool = False each of the given columns. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation count_nulls (bool): whether null values should be counted, default is False @@ -161,7 +170,7 @@ def first(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a First aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -171,31 +180,48 @@ def first(cols: Union[str, List[str]] = None) -> Aggregation: return _AggregationColumns(agg_spec=agg_spec, cols=to_list(cols)) -def formula(formula: str, formula_param: str, cols: Union[str, List[str]] = None) -> Aggregation: +def formula(formula: str, formula_param: Optional[str] = None, cols: Optional[Union[str, List[str]]] = None) -> Aggregation: """Creates a user defined formula aggregation. This formula can contain a combination of any of the following: | Built-in functions such as `min`, `max`, etc. | Mathematical arithmetic such as `*`, `+`, `/`, etc. | User-defined functions + There are two variants of this call. The preferred variant requires the formula to provide the output column name + and specific input column names in the following format: + | formula('output_col=(input_col1 + input_col2) * input_col3') + This form does not accept `formula_param` or `cols` arguments because the input and output columns are explicitly + set within the formula string. + + The second (deprecated) variant allows the user to apply a formula expression to one input column, producing one + output column. In this call the `formula_param` is used as a placeholder for the input column name and the `cols` + argument is used to identify the output column name and the input source column when applying the formula. If + multiple input/output pairs are specified in the `cols` argument, the formula will be applied to each column in the + list. + Args: - formula (str): the user defined formula to apply to each group. - formula_param (str): the parameter name for the input column's vector within the formula. If formula is - `max(each)`, then `each` is the formula_param. - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; - default is None, only valid when used in Table agg_all_by operation + formula (str): the user defined formula to apply + formula_param (Optional[str]): If provided, supplies the parameter name for the input column's vector within the + formula. If formula is `max(each)`, then `each` should be the formula_param. This must be set to None (the + default when omitted) when the `formula`argument specifies the input and output columns. + cols (Optional[Union[str, List[str]]]): If provided, supplies the column(s) to aggregate, can be renaming + expressions, i.e. "new_col = col". This must be set to None (the default when omitted) when the `formula` + argument specifies the input and output columns. Returns: an aggregation """ - agg_spec = _GrpcAggSpec(formula=_GrpcAggSpec.AggSpecFormula(formula=formula, param_token=formula_param)) - return _AggregationColumns(agg_spec=agg_spec, cols=to_list(cols)) + if formula_param: + agg_spec = _GrpcAggSpec(formula=_GrpcAggSpec.AggSpecFormula(formula=formula, param_token=formula_param)) + return _AggregationColumns(agg_spec=agg_spec, cols=to_list(cols)) + selectable = _GrpcSelectable(raw=formula) + return _AggregationFormula(selectable=selectable) def last(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Last aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -209,7 +235,7 @@ def min_(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Min aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -223,7 +249,7 @@ def max_(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Max aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -238,7 +264,7 @@ def median(cols: Union[str, List[str]] = None, average_evenly_divided: bool = Tr given columns. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation average_evenly_divided (bool): when the group size is an even number, whether to average the two middle values for the output value. When set to True, average the two middle values. When set to False, use the smaller @@ -257,7 +283,7 @@ def pct(percentile: float, cols: Union[str, List[str]] = None, average_evenly_di Args: percentile (float): the percentile used for calculation - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation average_evenly_divided (bool): when the percentile splits the group into two halves, whether to average the two middle values for the output value. When set to True, average the two middle values. When set to False, use @@ -276,7 +302,7 @@ def sorted_first(order_by: str, cols: Union[str, List[str]] = None) -> Aggregati Args: order_by (str): the column to sort by - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -292,7 +318,7 @@ def sorted_last(order_by: str, cols: Union[str, List[str]] = None) -> Aggregatio Args: order_by (str): the column to sort by - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -310,7 +336,7 @@ def std(cols: Union[str, List[str]] = None) -> Aggregation: which ensures that the sample variance will be an unbiased estimator of population variance. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -327,7 +353,7 @@ def unique(cols: Union[str, List[str]] = None, include_nulls: bool = False, result is the specified non_unique_sentinel value (defaults to null). Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation include_nulls (bool): whether null is treated as a value for the purpose of determining if the values in the aggregation group are unique, default is False. @@ -379,7 +405,7 @@ def var(cols: Union[str, List[str]] = None) -> Aggregation: which ensures that the sample variance will be an unbiased estimator of population variance. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -394,7 +420,7 @@ def weighted_avg(wcol: str, cols: Union[str, List[str]] = None) -> Aggregation: Args: wcol (str): the name of the weight column - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -409,7 +435,7 @@ def weighted_sum(wcol: str, cols: Union[str, List[str]] = None) -> Aggregation: Args: wcol (str): the name of the weight column - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -424,7 +450,7 @@ def distinct(cols: Union[str, List[str]] = None, include_nulls: bool = False) -> given columns and stores them as vectors. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation include_nulls (bool): whether nulls should be included as distinct values, default is False diff --git a/py/client/pydeephaven/proto/table_pb2.py b/py/client/pydeephaven/proto/table_pb2.py index c2a1ac0528a..bb4d50020dc 100644 --- a/py/client/pydeephaven/proto/table_pb2.py +++ b/py/client/pydeephaven/proto/table_pb2.py @@ -15,7 +15,7 @@ from pydeephaven.proto import ticket_pb2 as deephaven_dot_proto_dot_ticket__pb2 -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x1b\x64\x65\x65phaven/proto/table.proto\x12!io.deephaven.proto.backplane.grpc\x1a\x1c\x64\x65\x65phaven/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\"\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\"\xd3\x05\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\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\x42\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\x1b\x64\x65\x65phaven/proto/table.proto\x12!io.deephaven.proto.backplane.grpc\x1a\x1c\x64\x65\x65phaven/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') _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=28943 - _globals['_BADDATABEHAVIOR']._serialized_end=29041 - _globals['_UPDATEBYNULLBEHAVIOR']._serialized_start=29043 - _globals['_UPDATEBYNULLBEHAVIOR']._serialized_end=29159 - _globals['_NULLVALUE']._serialized_start=29161 - _globals['_NULLVALUE']._serialized_end=29188 - _globals['_CASESENSITIVITY']._serialized_start=29190 - _globals['_CASESENSITIVITY']._serialized_end=29240 - _globals['_MATCHTYPE']._serialized_start=29242 - _globals['_MATCHTYPE']._serialized_end=29280 + _globals['_BADDATABEHAVIOR']._serialized_start=29155 + _globals['_BADDATABEHAVIOR']._serialized_end=29253 + _globals['_UPDATEBYNULLBEHAVIOR']._serialized_start=29255 + _globals['_UPDATEBYNULLBEHAVIOR']._serialized_end=29371 + _globals['_NULLVALUE']._serialized_start=29373 + _globals['_NULLVALUE']._serialized_end=29400 + _globals['_CASESENSITIVITY']._serialized_start=29402 + _globals['_CASESENSITIVITY']._serialized_end=29452 + _globals['_MATCHTYPE']._serialized_start=29454 + _globals['_MATCHTYPE']._serialized_end=29492 _globals['_TABLEREFERENCE']._serialized_start=96 _globals['_TABLEREFERENCE']._serialized_end=204 _globals['_EXPORTEDTABLECREATIONRESPONSE']._serialized_start=207 @@ -97,252 +97,256 @@ _globals['_TIMETABLEREQUEST']._serialized_end=1239 _globals['_SELECTORUPDATEREQUEST']._serialized_start=1242 _globals['_SELECTORUPDATEREQUEST']._serialized_end=1419 - _globals['_MATHCONTEXT']._serialized_start=1422 - _globals['_MATHCONTEXT']._serialized_end=1690 - _globals['_MATHCONTEXT_ROUNDINGMODE']._serialized_start=1541 - _globals['_MATHCONTEXT_ROUNDINGMODE']._serialized_end=1690 - _globals['_UPDATEBYWINDOWSCALE']._serialized_start=1693 - _globals['_UPDATEBYWINDOWSCALE']._serialized_end=2040 - _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTICKS']._serialized_start=1900 - _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTICKS']._serialized_end=1936 - _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTIME']._serialized_start=1938 - _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTIME']._serialized_end=2032 - _globals['_UPDATEBYEMOPTIONS']._serialized_start=2043 - _globals['_UPDATEBYEMOPTIONS']._serialized_end=2524 - _globals['_UPDATEBYDELTAOPTIONS']._serialized_start=2526 - _globals['_UPDATEBYDELTAOPTIONS']._serialized_end=2628 - _globals['_UPDATEBYREQUEST']._serialized_start=2631 - _globals['_UPDATEBYREQUEST']._serialized_end=9698 - _globals['_UPDATEBYREQUEST_UPDATEBYOPTIONS']._serialized_start=2984 - _globals['_UPDATEBYREQUEST_UPDATEBYOPTIONS']._serialized_end=3435 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_start=3438 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_end=9698 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_start=3563 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_end=9690 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_start=3715 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_end=9690 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_start=6618 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_end=6641 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_start=6643 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_end=6666 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_start=6668 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_end=6691 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_start=6693 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_end=6720 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_start=6722 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_end=6736 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_start=6739 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_end=6901 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_start=6904 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_end=7066 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_start=7069 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_end=7233 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_start=7236 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_end=7400 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_start=7403 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_end=7567 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_start=7569 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_end=7658 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_start=7661 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_end=7853 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_start=7856 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_end=8050 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_start=8053 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_end=8245 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_start=8248 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_end=8440 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_start=8443 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_end=8635 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_start=8638 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_end=8834 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_start=8837 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_end=9031 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_start=9034 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_end=9226 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_start=9229 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_end=9445 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_start=9448 - _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_end=9682 - _globals['_SELECTDISTINCTREQUEST']._serialized_start=9701 - _globals['_SELECTDISTINCTREQUEST']._serialized_end=9878 - _globals['_DROPCOLUMNSREQUEST']._serialized_start=9881 - _globals['_DROPCOLUMNSREQUEST']._serialized_end=10055 - _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_start=10058 - _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_end=10239 - _globals['_HEADORTAILREQUEST']._serialized_start=10242 - _globals['_HEADORTAILREQUEST']._serialized_end=10415 - _globals['_HEADORTAILBYREQUEST']._serialized_start=10418 - _globals['_HEADORTAILBYREQUEST']._serialized_end=10624 - _globals['_UNGROUPREQUEST']._serialized_start=10627 - _globals['_UNGROUPREQUEST']._serialized_end=10822 - _globals['_MERGETABLESREQUEST']._serialized_start=10825 - _globals['_MERGETABLESREQUEST']._serialized_end=10998 - _globals['_SNAPSHOTTABLEREQUEST']._serialized_start=11001 - _globals['_SNAPSHOTTABLEREQUEST']._serialized_end=11155 - _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_start=11158 - _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_end=11463 - _globals['_CROSSJOINTABLESREQUEST']._serialized_start=11466 - _globals['_CROSSJOINTABLESREQUEST']._serialized_end=11761 - _globals['_NATURALJOINTABLESREQUEST']._serialized_start=11764 - _globals['_NATURALJOINTABLESREQUEST']._serialized_end=12039 - _globals['_EXACTJOINTABLESREQUEST']._serialized_start=12042 - _globals['_EXACTJOINTABLESREQUEST']._serialized_end=12315 - _globals['_LEFTJOINTABLESREQUEST']._serialized_start=12318 - _globals['_LEFTJOINTABLESREQUEST']._serialized_end=12590 - _globals['_ASOFJOINTABLESREQUEST']._serialized_start=12593 - _globals['_ASOFJOINTABLESREQUEST']._serialized_end=13058 - _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_start=12961 - _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_end=13054 - _globals['_AJRAJTABLESREQUEST']._serialized_start=13061 - _globals['_AJRAJTABLESREQUEST']._serialized_end=13355 - _globals['_MULTIJOININPUT']._serialized_start=13358 - _globals['_MULTIJOININPUT']._serialized_end=13494 - _globals['_MULTIJOINTABLESREQUEST']._serialized_start=13497 - _globals['_MULTIJOINTABLESREQUEST']._serialized_end=13661 - _globals['_RANGEJOINTABLESREQUEST']._serialized_start=13664 - _globals['_RANGEJOINTABLESREQUEST']._serialized_end=14528 - _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_start=14285 - _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_end=14403 - _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_start=14405 - _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_end=14528 - _globals['_COMBOAGGREGATEREQUEST']._serialized_start=14531 - _globals['_COMBOAGGREGATEREQUEST']._serialized_end=15169 - _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_start=14824 - _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_end=14997 - _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_start=15000 - _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_end=15165 - _globals['_AGGREGATEALLREQUEST']._serialized_start=15172 - _globals['_AGGREGATEALLREQUEST']._serialized_end=15409 - _globals['_AGGSPEC']._serialized_start=15412 - _globals['_AGGSPEC']._serialized_end=18443 - _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_start=17218 - _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_end=17310 - _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_start=17312 - _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_end=17355 - _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_start=17357 - _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_end=17397 - _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_start=17399 - _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_end=17453 - _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_start=17455 - _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_end=17502 - _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_start=17504 - _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_end=17575 - _globals['_AGGSPEC_AGGSPECSORTED']._serialized_start=17577 - _globals['_AGGSPEC_AGGSPECSORTED']._serialized_end=17673 - _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_start=17675 - _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_end=17717 - _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_start=17719 - _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_end=17777 - _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_start=17780 - _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_end=17916 - _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_start=17919 - _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_end=18228 - _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_start=18230 - _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_end=18270 - _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_start=18272 - _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_end=18287 - _globals['_AGGSPEC_AGGSPECAVG']._serialized_start=18289 - _globals['_AGGSPEC_AGGSPECAVG']._serialized_end=18301 - _globals['_AGGSPEC_AGGSPECFIRST']._serialized_start=18303 - _globals['_AGGSPEC_AGGSPECFIRST']._serialized_end=18317 - _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_start=18319 - _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_end=18334 - _globals['_AGGSPEC_AGGSPECGROUP']._serialized_start=18336 - _globals['_AGGSPEC_AGGSPECGROUP']._serialized_end=18350 - _globals['_AGGSPEC_AGGSPECLAST']._serialized_start=18352 - _globals['_AGGSPEC_AGGSPECLAST']._serialized_end=18365 - _globals['_AGGSPEC_AGGSPECMAX']._serialized_start=18367 - _globals['_AGGSPEC_AGGSPECMAX']._serialized_end=18379 - _globals['_AGGSPEC_AGGSPECMIN']._serialized_start=18381 - _globals['_AGGSPEC_AGGSPECMIN']._serialized_end=18393 - _globals['_AGGSPEC_AGGSPECSTD']._serialized_start=18395 - _globals['_AGGSPEC_AGGSPECSTD']._serialized_end=18407 - _globals['_AGGSPEC_AGGSPECSUM']._serialized_start=18409 - _globals['_AGGSPEC_AGGSPECSUM']._serialized_end=18421 - _globals['_AGGSPEC_AGGSPECVAR']._serialized_start=18423 - _globals['_AGGSPEC_AGGSPECVAR']._serialized_end=18435 - _globals['_AGGREGATEREQUEST']._serialized_start=18446 - _globals['_AGGREGATEREQUEST']._serialized_end=18794 - _globals['_AGGREGATION']._serialized_start=18797 - _globals['_AGGREGATION']._serialized_end=19520 - _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_start=19251 - _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_end=19350 - _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_start=19352 - _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_end=19391 - _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_start=19393 - _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_end=19433 - _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_start=19435 - _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_end=19512 - _globals['_SORTDESCRIPTOR']._serialized_start=19523 - _globals['_SORTDESCRIPTOR']._serialized_end=19748 - _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_start=19667 - _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_end=19748 - _globals['_SORTTABLEREQUEST']._serialized_start=19751 - _globals['_SORTTABLEREQUEST']._serialized_end=19967 - _globals['_FILTERTABLEREQUEST']._serialized_start=19970 - _globals['_FILTERTABLEREQUEST']._serialized_end=20185 - _globals['_SEEKROWREQUEST']._serialized_start=20188 - _globals['_SEEKROWREQUEST']._serialized_end=20437 - _globals['_SEEKROWRESPONSE']._serialized_start=20439 - _globals['_SEEKROWRESPONSE']._serialized_end=20480 - _globals['_REFERENCE']._serialized_start=20482 - _globals['_REFERENCE']._serialized_end=20514 - _globals['_LITERAL']._serialized_start=20517 - _globals['_LITERAL']._serialized_end=20662 - _globals['_VALUE']._serialized_start=20665 - _globals['_VALUE']._serialized_end=20810 - _globals['_CONDITION']._serialized_start=20813 - _globals['_CONDITION']._serialized_end=21513 - _globals['_ANDCONDITION']._serialized_start=21515 - _globals['_ANDCONDITION']._serialized_end=21592 - _globals['_ORCONDITION']._serialized_start=21594 - _globals['_ORCONDITION']._serialized_end=21670 - _globals['_NOTCONDITION']._serialized_start=21672 - _globals['_NOTCONDITION']._serialized_end=21748 - _globals['_COMPARECONDITION']._serialized_start=21751 - _globals['_COMPARECONDITION']._serialized_end=22179 - _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_start=22049 - _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_end=22179 - _globals['_INCONDITION']._serialized_start=22182 - _globals['_INCONDITION']._serialized_end=22459 - _globals['_INVOKECONDITION']._serialized_start=22462 - _globals['_INVOKECONDITION']._serialized_end=22614 - _globals['_ISNULLCONDITION']._serialized_start=22616 - _globals['_ISNULLCONDITION']._serialized_end=22698 - _globals['_MATCHESCONDITION']._serialized_start=22701 - _globals['_MATCHESCONDITION']._serialized_end=22943 - _globals['_CONTAINSCONDITION']._serialized_start=22946 - _globals['_CONTAINSCONDITION']._serialized_end=23197 - _globals['_SEARCHCONDITION']._serialized_start=23199 - _globals['_SEARCHCONDITION']._serialized_end=23314 - _globals['_FLATTENREQUEST']._serialized_start=23317 - _globals['_FLATTENREQUEST']._serialized_end=23465 - _globals['_METATABLEREQUEST']._serialized_start=23468 - _globals['_METATABLEREQUEST']._serialized_end=23618 - _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_start=23621 - _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_end=24057 - _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_start=23942 - _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_end=24057 - _globals['_CREATEINPUTTABLEREQUEST']._serialized_start=24060 - _globals['_CREATEINPUTTABLEREQUEST']._serialized_end=24796 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_start=24335 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_end=24782 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_start=24703 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_end=24723 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_start=24725 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_end=24765 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_start=24767 - _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_end=24774 - _globals['_WHEREINREQUEST']._serialized_start=24799 - _globals['_WHEREINREQUEST']._serialized_end=25058 - _globals['_COLUMNSTATISTICSREQUEST']._serialized_start=25061 - _globals['_COLUMNSTATISTICSREQUEST']._serialized_end=25295 - _globals['_SLICEREQUEST']._serialized_start=25298 - _globals['_SLICEREQUEST']._serialized_end=25519 - _globals['_BATCHTABLEREQUEST']._serialized_start=25522 - _globals['_BATCHTABLEREQUEST']._serialized_end=28941 - _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_start=25621 - _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_end=28941 - _globals['_TABLESERVICE']._serialized_start=29283 - _globals['_TABLESERVICE']._serialized_end=35740 + _globals['_SELECTABLE']._serialized_start=1421 + _globals['_SELECTABLE']._serialized_end=1456 + _globals['_MATHCONTEXT']._serialized_start=1459 + _globals['_MATHCONTEXT']._serialized_end=1727 + _globals['_MATHCONTEXT_ROUNDINGMODE']._serialized_start=1578 + _globals['_MATHCONTEXT_ROUNDINGMODE']._serialized_end=1727 + _globals['_UPDATEBYWINDOWSCALE']._serialized_start=1730 + _globals['_UPDATEBYWINDOWSCALE']._serialized_end=2077 + _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTICKS']._serialized_start=1937 + _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTICKS']._serialized_end=1973 + _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTIME']._serialized_start=1975 + _globals['_UPDATEBYWINDOWSCALE_UPDATEBYWINDOWTIME']._serialized_end=2069 + _globals['_UPDATEBYEMOPTIONS']._serialized_start=2080 + _globals['_UPDATEBYEMOPTIONS']._serialized_end=2561 + _globals['_UPDATEBYDELTAOPTIONS']._serialized_start=2563 + _globals['_UPDATEBYDELTAOPTIONS']._serialized_end=2665 + _globals['_UPDATEBYREQUEST']._serialized_start=2668 + _globals['_UPDATEBYREQUEST']._serialized_end=9735 + _globals['_UPDATEBYREQUEST_UPDATEBYOPTIONS']._serialized_start=3021 + _globals['_UPDATEBYREQUEST_UPDATEBYOPTIONS']._serialized_end=3472 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_start=3475 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_end=9735 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_start=3600 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_end=9727 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_start=3752 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_end=9727 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_start=6655 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_end=6678 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_start=6680 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_end=6703 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_start=6705 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_end=6728 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_start=6730 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_end=6757 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_start=6759 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_end=6773 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_start=6776 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_end=6938 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_start=6941 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_end=7103 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_start=7106 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_end=7270 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_start=7273 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_end=7437 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_start=7440 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_end=7604 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_start=7606 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_end=7695 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_start=7698 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_end=7890 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_start=7893 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_end=8087 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_start=8090 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_end=8282 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_start=8285 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_end=8477 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_start=8480 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_end=8672 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_start=8675 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_end=8871 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_start=8874 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_end=9068 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_start=9071 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_end=9263 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_start=9266 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_end=9482 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_start=9485 + _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_end=9719 + _globals['_SELECTDISTINCTREQUEST']._serialized_start=9738 + _globals['_SELECTDISTINCTREQUEST']._serialized_end=9915 + _globals['_DROPCOLUMNSREQUEST']._serialized_start=9918 + _globals['_DROPCOLUMNSREQUEST']._serialized_end=10092 + _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_start=10095 + _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_end=10276 + _globals['_HEADORTAILREQUEST']._serialized_start=10279 + _globals['_HEADORTAILREQUEST']._serialized_end=10452 + _globals['_HEADORTAILBYREQUEST']._serialized_start=10455 + _globals['_HEADORTAILBYREQUEST']._serialized_end=10661 + _globals['_UNGROUPREQUEST']._serialized_start=10664 + _globals['_UNGROUPREQUEST']._serialized_end=10859 + _globals['_MERGETABLESREQUEST']._serialized_start=10862 + _globals['_MERGETABLESREQUEST']._serialized_end=11035 + _globals['_SNAPSHOTTABLEREQUEST']._serialized_start=11038 + _globals['_SNAPSHOTTABLEREQUEST']._serialized_end=11192 + _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_start=11195 + _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_end=11500 + _globals['_CROSSJOINTABLESREQUEST']._serialized_start=11503 + _globals['_CROSSJOINTABLESREQUEST']._serialized_end=11798 + _globals['_NATURALJOINTABLESREQUEST']._serialized_start=11801 + _globals['_NATURALJOINTABLESREQUEST']._serialized_end=12076 + _globals['_EXACTJOINTABLESREQUEST']._serialized_start=12079 + _globals['_EXACTJOINTABLESREQUEST']._serialized_end=12352 + _globals['_LEFTJOINTABLESREQUEST']._serialized_start=12355 + _globals['_LEFTJOINTABLESREQUEST']._serialized_end=12627 + _globals['_ASOFJOINTABLESREQUEST']._serialized_start=12630 + _globals['_ASOFJOINTABLESREQUEST']._serialized_end=13095 + _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_start=12998 + _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_end=13091 + _globals['_AJRAJTABLESREQUEST']._serialized_start=13098 + _globals['_AJRAJTABLESREQUEST']._serialized_end=13392 + _globals['_MULTIJOININPUT']._serialized_start=13395 + _globals['_MULTIJOININPUT']._serialized_end=13531 + _globals['_MULTIJOINTABLESREQUEST']._serialized_start=13534 + _globals['_MULTIJOINTABLESREQUEST']._serialized_end=13698 + _globals['_RANGEJOINTABLESREQUEST']._serialized_start=13701 + _globals['_RANGEJOINTABLESREQUEST']._serialized_end=14565 + _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_start=14322 + _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_end=14440 + _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_start=14442 + _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_end=14565 + _globals['_COMBOAGGREGATEREQUEST']._serialized_start=14568 + _globals['_COMBOAGGREGATEREQUEST']._serialized_end=15206 + _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_start=14861 + _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_end=15034 + _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_start=15037 + _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_end=15202 + _globals['_AGGREGATEALLREQUEST']._serialized_start=15209 + _globals['_AGGREGATEALLREQUEST']._serialized_end=15446 + _globals['_AGGSPEC']._serialized_start=15449 + _globals['_AGGSPEC']._serialized_end=18480 + _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_start=17255 + _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_end=17347 + _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_start=17349 + _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_end=17392 + _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_start=17394 + _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_end=17434 + _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_start=17436 + _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_end=17490 + _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_start=17492 + _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_end=17539 + _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_start=17541 + _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_end=17612 + _globals['_AGGSPEC_AGGSPECSORTED']._serialized_start=17614 + _globals['_AGGSPEC_AGGSPECSORTED']._serialized_end=17710 + _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_start=17712 + _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_end=17754 + _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_start=17756 + _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_end=17814 + _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_start=17817 + _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_end=17953 + _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_start=17956 + _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_end=18265 + _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_start=18267 + _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_end=18307 + _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_start=18309 + _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_end=18324 + _globals['_AGGSPEC_AGGSPECAVG']._serialized_start=18326 + _globals['_AGGSPEC_AGGSPECAVG']._serialized_end=18338 + _globals['_AGGSPEC_AGGSPECFIRST']._serialized_start=18340 + _globals['_AGGSPEC_AGGSPECFIRST']._serialized_end=18354 + _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_start=18356 + _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_end=18371 + _globals['_AGGSPEC_AGGSPECGROUP']._serialized_start=18373 + _globals['_AGGSPEC_AGGSPECGROUP']._serialized_end=18387 + _globals['_AGGSPEC_AGGSPECLAST']._serialized_start=18389 + _globals['_AGGSPEC_AGGSPECLAST']._serialized_end=18402 + _globals['_AGGSPEC_AGGSPECMAX']._serialized_start=18404 + _globals['_AGGSPEC_AGGSPECMAX']._serialized_end=18416 + _globals['_AGGSPEC_AGGSPECMIN']._serialized_start=18418 + _globals['_AGGSPEC_AGGSPECMIN']._serialized_end=18430 + _globals['_AGGSPEC_AGGSPECSTD']._serialized_start=18432 + _globals['_AGGSPEC_AGGSPECSTD']._serialized_end=18444 + _globals['_AGGSPEC_AGGSPECSUM']._serialized_start=18446 + _globals['_AGGSPEC_AGGSPECSUM']._serialized_end=18458 + _globals['_AGGSPEC_AGGSPECVAR']._serialized_start=18460 + _globals['_AGGSPEC_AGGSPECVAR']._serialized_end=18472 + _globals['_AGGREGATEREQUEST']._serialized_start=18483 + _globals['_AGGREGATEREQUEST']._serialized_end=18831 + _globals['_AGGREGATION']._serialized_start=18834 + _globals['_AGGREGATION']._serialized_end=19732 + _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_start=19374 + _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_end=19473 + _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_start=19475 + _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_end=19514 + _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_start=19516 + _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_end=19556 + _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_start=19558 + _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_end=19635 + _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_start=19637 + _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_end=19724 + _globals['_SORTDESCRIPTOR']._serialized_start=19735 + _globals['_SORTDESCRIPTOR']._serialized_end=19960 + _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_start=19879 + _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_end=19960 + _globals['_SORTTABLEREQUEST']._serialized_start=19963 + _globals['_SORTTABLEREQUEST']._serialized_end=20179 + _globals['_FILTERTABLEREQUEST']._serialized_start=20182 + _globals['_FILTERTABLEREQUEST']._serialized_end=20397 + _globals['_SEEKROWREQUEST']._serialized_start=20400 + _globals['_SEEKROWREQUEST']._serialized_end=20649 + _globals['_SEEKROWRESPONSE']._serialized_start=20651 + _globals['_SEEKROWRESPONSE']._serialized_end=20692 + _globals['_REFERENCE']._serialized_start=20694 + _globals['_REFERENCE']._serialized_end=20726 + _globals['_LITERAL']._serialized_start=20729 + _globals['_LITERAL']._serialized_end=20874 + _globals['_VALUE']._serialized_start=20877 + _globals['_VALUE']._serialized_end=21022 + _globals['_CONDITION']._serialized_start=21025 + _globals['_CONDITION']._serialized_end=21725 + _globals['_ANDCONDITION']._serialized_start=21727 + _globals['_ANDCONDITION']._serialized_end=21804 + _globals['_ORCONDITION']._serialized_start=21806 + _globals['_ORCONDITION']._serialized_end=21882 + _globals['_NOTCONDITION']._serialized_start=21884 + _globals['_NOTCONDITION']._serialized_end=21960 + _globals['_COMPARECONDITION']._serialized_start=21963 + _globals['_COMPARECONDITION']._serialized_end=22391 + _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_start=22261 + _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_end=22391 + _globals['_INCONDITION']._serialized_start=22394 + _globals['_INCONDITION']._serialized_end=22671 + _globals['_INVOKECONDITION']._serialized_start=22674 + _globals['_INVOKECONDITION']._serialized_end=22826 + _globals['_ISNULLCONDITION']._serialized_start=22828 + _globals['_ISNULLCONDITION']._serialized_end=22910 + _globals['_MATCHESCONDITION']._serialized_start=22913 + _globals['_MATCHESCONDITION']._serialized_end=23155 + _globals['_CONTAINSCONDITION']._serialized_start=23158 + _globals['_CONTAINSCONDITION']._serialized_end=23409 + _globals['_SEARCHCONDITION']._serialized_start=23411 + _globals['_SEARCHCONDITION']._serialized_end=23526 + _globals['_FLATTENREQUEST']._serialized_start=23529 + _globals['_FLATTENREQUEST']._serialized_end=23677 + _globals['_METATABLEREQUEST']._serialized_start=23680 + _globals['_METATABLEREQUEST']._serialized_end=23830 + _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_start=23833 + _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_end=24269 + _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_start=24154 + _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_end=24269 + _globals['_CREATEINPUTTABLEREQUEST']._serialized_start=24272 + _globals['_CREATEINPUTTABLEREQUEST']._serialized_end=25008 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_start=24547 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_end=24994 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_start=24915 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_end=24935 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_start=24937 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_end=24977 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_start=24979 + _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_end=24986 + _globals['_WHEREINREQUEST']._serialized_start=25011 + _globals['_WHEREINREQUEST']._serialized_end=25270 + _globals['_COLUMNSTATISTICSREQUEST']._serialized_start=25273 + _globals['_COLUMNSTATISTICSREQUEST']._serialized_end=25507 + _globals['_SLICEREQUEST']._serialized_start=25510 + _globals['_SLICEREQUEST']._serialized_end=25731 + _globals['_BATCHTABLEREQUEST']._serialized_start=25734 + _globals['_BATCHTABLEREQUEST']._serialized_end=29153 + _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_start=25833 + _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_end=29153 + _globals['_TABLESERVICE']._serialized_start=29495 + _globals['_TABLESERVICE']._serialized_end=35952 # @@protoc_insertion_point(module_scope) diff --git a/py/client/tests/test_table.py b/py/client/tests/test_table.py index 58dbd29b622..25d4067a87a 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 +from pydeephaven.agg import sum_, avg, pct, weighted_avg, count_, partition, median, unique, count_distinct, distinct, formula from pydeephaven.table import Table from tests.testbase import BaseTestCase @@ -248,6 +248,12 @@ def test_agg_by(self): weighted_avg(wcol="d", cols=["WavGD = d"]), count_(col="ca"), 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"]), + formula(formula="f_const=5.0 + 3"), + formula(formula="f_min=min(a)"), + formula(formula="f_sum=sum(a) + sum(b)"), + formula(formula="f_sum_3col=sum(a) + sum(b) + max(c)"), ] result_table = test_table.agg_by(aggs=aggs, by=["a"]) diff --git a/py/server/deephaven/agg.py b/py/server/deephaven/agg.py index a8af429fc6a..222329f7803 100644 --- a/py/server/deephaven/agg.py +++ b/py/server/deephaven/agg.py @@ -3,7 +3,7 @@ # """This module implement various aggregations that can be used in deephaven table's aggregation operations.""" -from typing import List, Union, Any +from typing import List, Union, Any, Optional import jpy @@ -52,7 +52,7 @@ def sum_(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Sum aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -65,7 +65,7 @@ def abs_sum(cols: Union[str, List[str]] = None) -> Aggregation: """Creates an Absolute-sum aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -78,7 +78,7 @@ def group(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Group aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -91,7 +91,7 @@ def avg(cols: Union[str, List[str]] = None) -> Aggregation: """Creates an Average aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -134,7 +134,7 @@ def count_distinct(cols: Union[str, List[str]] = None, count_nulls: bool = False each of the given columns. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation count_nulls (bool): whether null values should be counted, default is False @@ -148,7 +148,7 @@ def first(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a First aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -157,30 +157,48 @@ def first(cols: Union[str, List[str]] = None) -> Aggregation: return Aggregation(j_agg_spec=_JAggSpec.first(), cols=cols) -def formula(formula: str, formula_param: str, cols: Union[str, List[str]] = None) -> Aggregation: +def formula(formula: str, formula_param: Optional[str] = None, cols: Optional[Union[str, List[str]]] = None) -> Aggregation: """Creates a user defined formula aggregation. This formula can contain a combination of any of the following: | Built-in functions such as `min`, `max`, etc. | Mathematical arithmetic such as `*`, `+`, `/`, etc. | User-defined functions + There are two variants of this call. The preferred variant requires the formula to provide the output column name + and specific input column names in the following format: + | formula('output_col=(input_col1 + input_col2) * input_col3') + This form does not accept `formula_param` or `cols` arguments because the input and output columns are explicitly + set within the formula string. + + The second (deprecated) variant allows the user to apply a formula expression to one input column, producing one + output column. In this call the `formula_param` is used as a placeholder for the input column name and the `cols` + argument is used to identify the output column name and the input source column when applying the formula. If + multiple input/output pairs are specified in the `cols` argument, the formula will be applied to each column in the + list. + Args: - formula (str): the user defined formula to apply to each group. - formula_param (str): the parameter name for the input column's vector within the formula. If formula is - `max(each)`, then `each` is the formula_param. - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; - default is None, only valid when used in Table agg_all_by operation + formula (str): the user defined formula to apply + formula_param (Optional[str]): If provided, supplies the parameter name for the input column's vector within the + formula. If formula is `max(each)`, then `each` should be the formula_param. This must be set to None (the + default when omitted) when the `formula`argument specifies the input and output columns. + cols (Optional[Union[str, List[str]]]): If provided, supplies the column(s) to aggregate, can be renaming + expressions, i.e. "new_col = col". This must be set to None (the default when omitted) when the `formula` + argument specifies the input and output columns. Returns: an aggregation """ - return Aggregation(j_agg_spec=_JAggSpec.formula(formula, formula_param), cols=cols) + if formula_param: + return Aggregation(j_agg_spec=_JAggSpec.formula(formula, formula_param), cols=cols) + if cols: + raise DHError(message="The 'cols' argument is only valid when 'formula_param' is provided.") + return Aggregation(j_aggregation=_JAggregation.AggFormula(formula)) def last(cols: Union[str, List[str]] = None) -> Aggregation: """Creates Last aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -193,7 +211,7 @@ def min_(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Min aggregation. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -206,7 +224,7 @@ def max_(cols: Union[str, List[str]] = None) -> Aggregation: """Creates a Max aggregation to the ComboAggregation object. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -220,7 +238,7 @@ def median(cols: Union[str, List[str]] = None, average_evenly_divided: bool = Tr given columns. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation average_evenly_divided (bool): when the group size is an even number, whether to average the two middle values for the output value. When set to True, average the two middle values. When set to False, use the smaller @@ -238,7 +256,7 @@ def pct(percentile: float, cols: Union[str, List[str]] = None, average_evenly_di Args: percentile (float): the percentile used for calculation - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation average_evenly_divided (bool): when the percentile splits the group into two halves, whether to average the two middle values for the output value. When set to True, average the two middle values. When set to False, use @@ -255,7 +273,7 @@ def sorted_first(order_by: str, cols: Union[str, List[str]] = None) -> Aggregati Args: order_by (str): the column to sort by - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -269,7 +287,7 @@ def sorted_last(order_by: str, cols: Union[str, List[str]] = None) -> Aggregatio Args: order_by (str): the column to sort by - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -286,7 +304,7 @@ def std(cols: Union[str, List[str]] = None) -> Aggregation: Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -301,7 +319,7 @@ def unique(cols: Union[str, List[str]] = None, include_nulls: bool = False, non_ result is null or the specified non_unique_sentinel value. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation include_nulls (bool): whether null is treated as a value for the purpose of determining if the values in the aggregation group are unique, default is False. @@ -330,7 +348,7 @@ def var(cols: Union[str, List[str]] = None) -> Aggregation: Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -344,7 +362,7 @@ def weighted_avg(wcol: str, cols: Union[str, List[str]] = None) -> Aggregation: Args: wcol (str): the name of the weight column - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -358,7 +376,7 @@ def weighted_sum(wcol: str, cols: Union[str, List[str]] = None) -> Aggregation: Args: wcol (str): the name of the weight column - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation Returns: @@ -372,7 +390,7 @@ def distinct(cols: Union[str, List[str]] = None, include_nulls: bool = False) -> given columns and stores them as vectors. Args: - cols (Union[str, List[str]]): the column(s) to aggregate on, can be renaming expressions, i.e. "new_col = col"; + cols (Union[str, List[str]]): the column(s) to aggregate, can be renaming expressions, i.e. "new_col = col"; default is None, only valid when used in Table agg_all_by operation include_nulls (bool): whether nulls should be included as distinct values, default is False diff --git a/py/server/tests/test_table.py b/py/server/tests/test_table.py index eba1e3ad8ca..70854c5d90a 100644 --- a/py/server/tests/test_table.py +++ b/py/server/tests/test_table.py @@ -461,6 +461,10 @@ def test_agg_by(self): formula( formula="min(each)", formula_param="each", cols=["MinA=a", "MinD=d"] ), + formula(formula="f_const=5.0 + 3"), + formula(formula="f_min=min(a)"), + formula(formula="f_sum=sum(a) + sum(b)"), + formula(formula="f_sum_3_col=sum(a) + sum(b) + max(c)"), ] result_table = self.test_table.agg_by(aggs=aggs, by=["a"]) 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 31b77003cc1..00eff2b9754 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 @@ -12,14 +12,17 @@ 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.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.AggregationFormula; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition; import io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey; import io.deephaven.proto.backplane.grpc.Aggregation.TypeCase; +import io.deephaven.proto.backplane.grpc.Selectable; import io.deephaven.proto.util.Exceptions; import io.deephaven.server.grpc.GrpcErrorHelper; @@ -59,6 +62,16 @@ public static void validate(io.deephaven.proto.backplane.grpc.Aggregation aggreg Singleton.INSTANCE.adapters().validate(aggregation); } + private static io.deephaven.api.Selectable adapt(Selectable selectable) { + switch (selectable.getTypeCase()) { + case RAW: + return io.deephaven.api.Selectable.parse(selectable.getRaw()); + default: + throw new IllegalArgumentException(String.format("Unsupported Selectable type (%s) in Aggregation.", + selectable.getTypeCase())); + } + } + public static Aggregation adapt(io.deephaven.proto.backplane.grpc.Aggregation aggregation) { return Singleton.INSTANCE.adapters().adapt(aggregation); } @@ -79,6 +92,10 @@ public static Count adapt(AggregationCount count) { return Aggregation.AggCount(count.getColumnName()); } + public static Formula adapt(AggregationFormula formula) { + return Formula.of(adapt(formula.getSelectable())); + } + public static FirstRowKey adaptFirst(AggregationRowKey key) { return Aggregation.AggFirstRowKey(key.getColumnName()); } @@ -194,6 +211,16 @@ public void visit(Partition partition) { AggregationAdapter::adapt); } + @Override + public void visit(Formula formula) { + add( + TypeCase.FORMULA, + AggregationFormula.class, + Formula.class, + GrpcErrorHelper::checkHasNoUnknownFieldsRecursive, + AggregationAdapter::adapt); + } + // New types _can_ be added as unimplemented. // If so, please create and link to a ticket. // @Override diff --git a/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java b/table-api/src/main/java/io/deephaven/api/agg/Aggregation.java index 3e5467c5c4c..66050567636 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 @@ -291,6 +291,40 @@ static FirstRowKey AggFirstRowKey(String resultColumn) { return FirstRowKey.of(resultColumn); } + /** + *

+ * Create a {@link Formula formula} aggregation with the supplied {@code formula}. This variant requires the formula + * to provide the output column name and specific input column names in the following format: + *

+ * {@code + * AggFormula("output_col=(input_col1 + input_col2) * input_col3") + * } + * + * @param formulaString The formula to use to produce the output column + * @return The aggregation + */ + static Formula AggFormula(String formulaString) { + return Formula.parse(formulaString); + } + + /** + *

+ * Create a {@link Formula formula} aggregation with the supplied {@code columnName} and {@code expression}. This + * variant requires the formula to provide the output column name and the expression to evaluate in the following + * format: + *

+ * {@code + * AggFormula("output_col", "(input_col1 + input_col2) * input_col3") + * } + * + * @param columnName The output column name + * @param expression The expression to use to produce the output column + * @return The aggregation + */ + static Formula AggFormula(String columnName, String expression) { + return Formula.of(columnName, expression); + } + /** * Create a {@link io.deephaven.api.agg.spec.AggSpecFormula formula} aggregation with the supplied {@code formula}, * {@code paramToken}, and column name pairs. @@ -302,6 +336,7 @@ static FirstRowKey AggFirstRowKey(String resultColumn) { * @param pairs The input/output column name pairs * @return The aggregation */ + @Deprecated static Aggregation AggFormula(String formula, String paramToken, String... pairs) { return of(AggSpec.formula(formula, paramToken), pairs); } @@ -678,6 +713,7 @@ static void visitAll(Visitor visitor) { visitor.visit((FirstRowKey) null); visitor.visit((LastRowKey) null); visitor.visit((Partition) null); + visitor.visit((Formula) null); } /** @@ -742,5 +778,12 @@ interface Visitor { * @param partition The partition aggregation */ void visit(Partition partition); + + /** + * Visit a {@link Formula formula aggregation}. + * + * @param formula The formula aggregation + */ + void visit(Formula formula); } } 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 dd7b991a809..ad435397fea 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 @@ -4,6 +4,7 @@ package io.deephaven.api.agg; import io.deephaven.api.Pair; +import io.deephaven.api.Strings; import java.util.Collection; import java.util.LinkedHashMap; @@ -79,4 +80,9 @@ public void visit(Partition partition) { out.put(partition.column().name(), "partition sub-table" + (partition.includeGroupByColumns() ? " (including group-by columns)" : "")); } + + @Override + public void visit(Formula formula) { + out.put(formula.column().name(), "from formula `" + Strings.of(formula.expression()) + "`"); + } } 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 949bcb1724c..b038de3217d 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 @@ -111,4 +111,10 @@ public void visit(Partition partition) { visitOrder.computeIfAbsent(partition.includeGroupByColumns() ? PARTITION_KEEPING_OBJ : PARTITION_DROPPING_OBJ, k -> new ArrayList<>()).add(partition.column()); } + + @Override + public void visit(Formula formula) { + // Supplying a `null` entry value indicates that the key is already an aggregation. + visitOrder.putIfAbsent(formula, null); + } } 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 2cd5d8990af..f40acae6163 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 @@ -63,4 +63,10 @@ public void visit(LastRowKey lastRowKey) { public void visit(Partition partition) { out = Stream.of(partition.column()); } + + + @Override + public void visit(Formula formula) { + out = Stream.of(formula.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 c3c67c716a4..2087e4894af 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 @@ -63,4 +63,10 @@ public void visit(LastRowKey lastRowKey) { public void visit(Partition partition) { out = Stream.empty(); } + + @Override + public void visit(Formula formula) { + out = Stream.empty(); + } + } diff --git a/table-api/src/main/java/io/deephaven/api/agg/Formula.java b/table-api/src/main/java/io/deephaven/api/agg/Formula.java new file mode 100644 index 00000000000..3b3bdf49385 --- /dev/null +++ b/table-api/src/main/java/io/deephaven/api/agg/Formula.java @@ -0,0 +1,62 @@ +// +// 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.RawString; +import io.deephaven.api.Selectable; +import io.deephaven.api.Strings; +import io.deephaven.api.expression.Expression; +import org.immutables.value.Value.Immutable; +import org.immutables.value.Value.Parameter; + +/** + * An {@link Aggregation aggregation} that provides a single output column that is computed by applying a formula to a + * set of input columns. + */ +@Immutable +@SimpleStyle +public abstract class Formula implements Aggregation { + + public static Formula parse(String formulaString) { + return ImmutableFormula.of(Selectable.parse(formulaString)); + } + + public static Formula of(String name, String formula) { + return of(ColumnName.of(name), formula); + } + + public static Formula of(ColumnName name, String formula) { + return of(Selectable.of(name, RawString.of(formula))); + } + + public static Formula of(Selectable selectable) { + return ImmutableFormula.of(selectable); + } + + @Parameter + public abstract Selectable selectable(); + + public ColumnName column() { + return selectable().newColumn(); + } + + public Expression expression() { + return selectable().expression(); + } + + /** + * Return this {@link Formula} as a string of the form {@code =} + */ + public String formulaString() { + return Strings.of(selectable()); + } + + @Override + public final V walk(V visitor) { + visitor.visit(this); + return visitor; + } +} diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java index 2a434927ee3..bb3609838a5 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpec.java @@ -79,16 +79,10 @@ static AggSpecFirst first() { return AggSpecFirst.of(); } - /** - * @return {@link AggSpecFormula#of(String)} for {@code formula} - */ - static AggSpecFormula formula(String formula) { - return AggSpecFormula.of(formula); - } - /** * @return {@link AggSpecFormula#of(String, String)} for {@code formula} and {@code paramToken} */ + @Deprecated static AggSpecFormula formula(String formula, String paramToken) { return AggSpecFormula.of(formula, paramToken); } diff --git a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java index de6662e6469..469e6b8f191 100644 --- a/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java +++ b/table-api/src/main/java/io/deephaven/api/agg/spec/AggSpecFormula.java @@ -15,6 +15,7 @@ */ @Immutable @SimpleStyle +@Deprecated public abstract class AggSpecFormula extends AggSpecBase { public static final String PARAM_TOKEN_DEFAULT = "each"; 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 963f59e367a..ad9a0f3a3bc 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 @@ -70,5 +70,10 @@ public void visit(LastRowKey lastRowKey) { public void visit(Partition partition) { ++count; } + + @Override + public void visit(Formula formula) { + ++count; + } } }