diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java
index ab36bbb757e..00388296aa0 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperator.java
@@ -33,7 +33,7 @@
* {@link UpdateByOperator#initializeRolling(Context, RowSet)} (Context)} for windowed operators
*
{@link UpdateByOperator.Context#accumulateCumulative(RowSequence, Chunk[], LongChunk, int)} for cumulative
* operators or
- * {@link UpdateByOperator.Context#accumulateRolling(RowSequence, Chunk[], LongChunk, LongChunk, IntChunk, IntChunk, int)}
+ * {@link UpdateByOperator.Context#accumulateRolling(RowSequence, Chunk[], LongChunk, LongChunk, IntChunk, IntChunk, int, int)}
* for windowed operators
* {@link #finishUpdate(UpdateByOperator.Context)}
*
@@ -99,18 +99,48 @@ protected void pop(int count) {
throw new UnsupportedOperationException("pop() must be overriden by rolling operators");
}
- public abstract void accumulateCumulative(RowSequence inputKeys,
+ /**
+ * For cumulative operators only, this method will be called to pass the input chunk data to the operator and
+ * produce the output data values.
+ *
+ * @param inputKeys the keys for the input data rows (also matches the output keys)
+ * @param valueChunkArr the input data chunks needed by the operator for internal calculations
+ * @param tsChunk the timestamp chunk for the input data (if applicable)
+ * @param len the number of items in the input data chunks
+ */
+ public abstract void accumulateCumulative(
+ RowSequence inputKeys,
Chunk extends Values>[] valueChunkArr,
LongChunk extends Values> tsChunk,
int len);
- public abstract void accumulateRolling(RowSequence inputKeys,
+ /**
+ * For windowed operators only, this method will be called to pass the input chunk data to the operator and
+ * produce the output data values. It is important to note that the size of the influencer (input) and affected
+ * (output) chunks are not likely be the same. We pass these sizes explicitly to the operators for the sake of
+ * the operators (such as {@link io.deephaven.engine.table.impl.updateby.countwhere.CountWhereOperator} with
+ * zero input columns) where no input chunks are provided but we must still process the exact number of input
+ * rows.
+ *
+ * @param inputKeys the keys for the input data rows (also matches the output keys)
+ * @param influencerValueChunkArr the input data chunks needed by the operator for internal calculations, these
+ * values will be pushed and popped into the current window
+ * @param affectedPosChunk the row positions of the affected rows
+ * @param influencerPosChunk the row positions of the influencer rows
+ * @param pushChunk a chunk containing the push instructions for each output row to be calculated
+ * @param popChunk a chunk containing the pop instructions for each output row to be calculated
+ * @param affectedCount how many affected (output) rows are being computed
+ * @param influencerCount how many influencer (input) rows are needed for the computation
+ */
+ public abstract void accumulateRolling(
+ RowSequence inputKeys,
Chunk extends Values>[] influencerValueChunkArr,
LongChunk affectedPosChunk,
LongChunk influencerPosChunk,
IntChunk extends Values> pushChunk,
IntChunk extends Values> popChunk,
- int len);
+ int affectedCount,
+ int influencerCount);
/**
* Write the current value for this row to the output chunk
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java
index 671e87bdd01..ad7e4687ffe 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByOperatorFactory.java
@@ -10,12 +10,21 @@
import io.deephaven.api.updateby.UpdateByControl;
import io.deephaven.api.updateby.UpdateByOperation;
import io.deephaven.api.updateby.spec.*;
+import io.deephaven.base.verify.Require;
+import io.deephaven.engine.rowset.RowSetFactory;
import io.deephaven.engine.table.ColumnDefinition;
+import io.deephaven.engine.table.ColumnSource;
+import io.deephaven.engine.table.Table;
import io.deephaven.engine.table.TableDefinition;
import io.deephaven.engine.table.impl.MatchPair;
import io.deephaven.engine.table.impl.QueryCompilerRequestProcessor;
+import io.deephaven.engine.table.impl.QueryTable;
import io.deephaven.engine.table.impl.select.FormulaColumn;
import io.deephaven.engine.table.impl.select.SelectColumn;
+import io.deephaven.engine.table.impl.select.WhereFilter;
+import io.deephaven.engine.table.impl.sources.NullValueColumnSource;
+import io.deephaven.engine.table.impl.sources.ReinterpretUtils;
+import io.deephaven.engine.table.impl.updateby.countwhere.CountWhereOperator;
import io.deephaven.engine.table.impl.updateby.delta.*;
import io.deephaven.engine.table.impl.updateby.em.*;
import io.deephaven.engine.table.impl.updateby.emstd.*;
@@ -45,6 +54,7 @@
import java.time.Instant;
import java.util.*;
import java.util.stream.Collectors;
+import java.util.stream.IntStream;
import java.util.stream.Stream;
import static io.deephaven.util.BooleanUtils.NULL_BOOLEAN_AS_BYTE;
@@ -414,6 +424,12 @@ public Void visit(CumProdSpec cps) {
return null;
}
+ @Override
+ public Void visit(CumCountWhereSpec spec) {
+ ops.add(makeCountWhereOperator(tableDef, spec));
+ return null;
+ }
+
@Override
public Void visit(@NotNull final DeltaSpec spec) {
Arrays.stream(pairs)
@@ -537,6 +553,12 @@ public Void visit(@NotNull final RollingCountSpec spec) {
return null;
}
+ @Override
+ public Void visit(@NotNull final RollingCountWhereSpec spec) {
+ ops.add(makeCountWhereOperator(tableDef, spec));
+ return null;
+ }
+
@Override
public Void visit(@NotNull final RollingFormulaSpec spec) {
final boolean isTimeBased = spec.revWindowScale().isTimeBased();
@@ -1240,6 +1262,130 @@ private UpdateByOperator makeRollingCountOperator(@NotNull final MatchPair pair,
}
}
+ /**
+ * This is used for Cum/Rolling CountWhere operators
+ */
+ private UpdateByOperator makeCountWhereOperator(
+ @NotNull final TableDefinition tableDef,
+ @NotNull final UpdateBySpec spec) {
+
+ Require.eqTrue(spec instanceof CumCountWhereSpec || spec instanceof RollingCountWhereSpec,
+ "spec instanceof CumCountWhereSpec || spec instanceof RollingCountWhereSpec");
+
+ final boolean isCumulative = spec instanceof CumCountWhereSpec;
+
+ final WhereFilter[] whereFilters = isCumulative
+ ? WhereFilter.fromInternal(((CumCountWhereSpec) spec).filter())
+ : WhereFilter.fromInternal(((RollingCountWhereSpec) spec).filter());
+
+ final List inputColumnNameList = new ArrayList<>();
+ final Map inputColumnMap = new HashMap<>();
+ final List filterInputColumnIndicesList = new ArrayList<>();
+
+ // Verify all the columns in the where filters are present in the dummy table and valid for use.
+ for (final WhereFilter whereFilter : whereFilters) {
+ whereFilter.init(tableDef);
+ if (whereFilter.isRefreshing()) {
+ throw new UnsupportedOperationException("CountWhere does not support refreshing filters");
+ }
+
+ // Compute which input sources this filter will use.
+ final List filterColumnName = whereFilter.getColumns();
+ final int inputColumnCount = whereFilter.getColumns().size();
+ final int[] inputColumnIndices = new int[inputColumnCount];
+ for (int ii = 0; ii < inputColumnCount; ++ii) {
+ final String inputColumnName = filterColumnName.get(ii);
+ final int inputColumnIndex = inputColumnMap.computeIfAbsent(inputColumnName, k -> {
+ inputColumnNameList.add(inputColumnName);
+ return inputColumnNameList.size() - 1;
+ });
+ inputColumnIndices[ii] = inputColumnIndex;
+ }
+ filterInputColumnIndicesList.add(inputColumnIndices);
+ }
+
+ // Gather the input column type info and create a dummy table we can use to initialize filters.
+ final String[] inputColumnNames = inputColumnNameList.toArray(String[]::new);
+ final ColumnSource>[] originalColumnSources = new ColumnSource[inputColumnNames.length];
+ final ColumnSource>[] reinterpretedColumnSources = new ColumnSource[inputColumnNames.length];
+
+ final Map> columnSourceMap = new LinkedHashMap<>();
+ for (int i = 0; i < inputColumnNames.length; i++) {
+ final String col = inputColumnNames[i];
+ final ColumnDefinition> def = tableDef.getColumn(col);
+ // Create a representative column source of the correct type for the filter.
+ final ColumnSource> nullSource =
+ NullValueColumnSource.getInstance(def.getDataType(), def.getComponentType());
+ // Create a reinterpreted version of the column source.
+ final ColumnSource> maybeReinterpretedSource = ReinterpretUtils.maybeConvertToPrimitive(nullSource);
+ if (nullSource != maybeReinterpretedSource) {
+ originalColumnSources[i] = nullSource;
+ }
+ columnSourceMap.put(col, maybeReinterpretedSource);
+ reinterpretedColumnSources[i] = maybeReinterpretedSource;
+ }
+ final Table dummyTable = new QueryTable(RowSetFactory.empty().toTracking(), columnSourceMap);
+
+ final CountWhereOperator.CountFilter[] countFilters =
+ CountWhereOperator.CountFilter.createCountFilters(whereFilters, dummyTable,
+ filterInputColumnIndicesList);
+
+ // If any filter is ConditionFilter or ChunkFilter and uses a reinterpreted column, need to produce
+ // original-typed chunks.
+ final boolean originalChunksRequired = Arrays.asList(countFilters).stream()
+ .anyMatch(filter -> (filter.chunkFilter() != null || filter.conditionFilter() != null)
+ && IntStream.of(filter.inputColumnIndices())
+ .anyMatch(i -> originalColumnSources[i] != null));
+
+ // If any filter is a standard WhereFilter or we need to produce original-typed chunks, need a chunk source
+ // table.
+ final boolean chunkSourceTableRequired = originalChunksRequired ||
+ Arrays.asList(countFilters).stream().anyMatch(filter -> filter.whereFilter() != null);
+
+ // Create a new column pair with the same name for the left and right columns
+ final String columnName = isCumulative
+ ? ((CumCountWhereSpec) spec).column().name()
+ : ((RollingCountWhereSpec) spec).column().name();
+ final MatchPair pair = new MatchPair(columnName, columnName);
+
+ // Create and return the operator.
+ if (isCumulative) {
+ return new CountWhereOperator(
+ pair,
+ countFilters,
+ inputColumnNames,
+ originalColumnSources,
+ reinterpretedColumnSources,
+ chunkSourceTableRequired,
+ originalChunksRequired);
+ } else {
+ final RollingCountWhereSpec rs = (RollingCountWhereSpec) spec;
+
+ final String[] affectingColumns;
+ if (rs.revWindowScale().timestampCol() == null) {
+ affectingColumns = inputColumnNames;
+ } else {
+ affectingColumns = ArrayUtils.add(inputColumnNames, rs.revWindowScale().timestampCol());
+ }
+
+ final long prevWindowScaleUnits = rs.revWindowScale().getTimeScaleUnits();
+ final long fwdWindowScaleUnits = rs.fwdWindowScale().getTimeScaleUnits();
+
+ return new CountWhereOperator(
+ pair,
+ affectingColumns,
+ rs.revWindowScale().timestampCol(),
+ prevWindowScaleUnits,
+ fwdWindowScaleUnits,
+ countFilters,
+ inputColumnNames,
+ originalColumnSources,
+ reinterpretedColumnSources,
+ chunkSourceTableRequired,
+ originalChunksRequired);
+ }
+ }
+
private UpdateByOperator makeRollingStdOperator(@NotNull final MatchPair pair,
@NotNull final TableDefinition tableDef,
@NotNull final RollingStdSpec rs) {
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowRollingBase.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowRollingBase.java
index 37ccac96fbe..2077f38df42 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowRollingBase.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/UpdateByWindowRollingBase.java
@@ -204,7 +204,8 @@ void processWindowBucketOperatorSet(final UpdateByWindowBucketContext context,
influencePosChunk,
ctx.pushChunks[affectedChunkOffset],
ctx.popChunks[affectedChunkOffset],
- affectedChunkSize);
+ affectedChunkSize,
+ influencerCount);
}
affectedChunkOffset++;
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/countwhere/CountWhereOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/countwhere/CountWhereOperator.java
new file mode 100644
index 00000000000..04ae34e6f0e
--- /dev/null
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/countwhere/CountWhereOperator.java
@@ -0,0 +1,627 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.engine.table.impl.updateby.countwhere;
+
+import io.deephaven.base.ringbuffer.ByteRingBuffer;
+import io.deephaven.base.verify.Assert;
+import io.deephaven.base.verify.Require;
+import io.deephaven.chunk.*;
+import io.deephaven.chunk.attributes.Values;
+import io.deephaven.engine.rowset.*;
+import io.deephaven.engine.rowset.chunkattributes.OrderedRowKeys;
+import io.deephaven.engine.table.*;
+import io.deephaven.engine.table.impl.MatchPair;
+import io.deephaven.engine.table.impl.QueryTable;
+import io.deephaven.engine.table.impl.chunkfilter.ChunkFilter;
+import io.deephaven.engine.table.impl.select.AbstractConditionFilter;
+import io.deephaven.engine.table.impl.select.ConditionFilter;
+import io.deephaven.engine.table.impl.select.ExposesChunkFilter;
+import io.deephaven.engine.table.impl.select.WhereFilter;
+import io.deephaven.engine.table.impl.sources.ReinterpretUtils;
+import io.deephaven.engine.table.impl.sources.chunkcolumnsource.ChunkColumnSource;
+import io.deephaven.engine.table.impl.updateby.UpdateByOperator;
+import io.deephaven.engine.table.impl.updateby.internal.BaseLongUpdateByOperator;
+import io.deephaven.util.SafeCloseable;
+import io.deephaven.util.SafeCloseableList;
+import org.apache.commons.lang3.ArrayUtils;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.*;
+
+import static io.deephaven.util.QueryConstants.*;
+
+public class CountWhereOperator extends BaseLongUpdateByOperator {
+ private static final int BUFFER_INITIAL_CAPACITY = 512;
+
+ /**
+ * Store the input column names and type information.
+ */
+ private final String[] inputColumnNames;
+ /**
+ * Store representative sources for reinterpreted input and original columns, will not hold data.
+ */
+ private final ColumnSource>[] originalSources;
+ private final ColumnSource>[] reinterpretedSources;
+ /**
+ * The raw filters to apply to the data.
+ */
+ private final CountFilter[] filters;
+
+ /**
+ * Whether we need to create and maintain a chunk source table.
+ */
+ private final boolean chunkSourceTableRequired;
+
+ /**
+ * Whether we need to produce original-typed chunks for condition and chunk filters.
+ */
+ private final boolean originalChunksRequired;
+
+ /**
+ * Holder class to hold filter objects and the input column indices they apply to.
+ */
+ public static class CountFilter {
+ final ChunkFilter chunkFilter;
+ final AbstractConditionFilter.Filter conditionFilter;
+ final WhereFilter whereFilter;
+ final int[] inputColumnIndices;
+
+ public CountFilter(ChunkFilter chunkFilter, int[] inputColumnIndices) {
+ this.chunkFilter = chunkFilter;
+ this.conditionFilter = null;
+ this.whereFilter = null;
+ this.inputColumnIndices = inputColumnIndices;
+ }
+
+ public CountFilter(AbstractConditionFilter.Filter conditionFilter, int[] inputColumnIndices) {
+ this.chunkFilter = null;
+ this.conditionFilter = conditionFilter;
+ this.whereFilter = null;
+ this.inputColumnIndices = inputColumnIndices;
+ }
+
+ public CountFilter(WhereFilter whereFilter, int[] inputColumnIndices) {
+ this.chunkFilter = null;
+ this.conditionFilter = null;
+ this.whereFilter = whereFilter;
+ this.inputColumnIndices = inputColumnIndices;
+ }
+
+ public ChunkFilter chunkFilter() {
+ return chunkFilter;
+ }
+
+ public AbstractConditionFilter.Filter conditionFilter() {
+ return conditionFilter;
+ }
+
+ public WhereFilter whereFilter() {
+ return whereFilter;
+ }
+
+ public int[] inputColumnIndices() {
+ return inputColumnIndices;
+ }
+
+ /**
+ * Create CountFilters from WhereFilter array, initializing the filters against the provided table.
+ */
+ public static CountFilter[] createCountFilters(
+ final WhereFilter[] filters,
+ final Table inputTable,
+ final List filterInputIndices) {
+
+ // Create the internal filters
+ final List filterList = new ArrayList<>();
+ boolean forcedWhereFilter = false;
+ for (int fi = 0; fi < filters.length; fi++) {
+ final WhereFilter filter = filters[fi];
+ final CountFilter countFilter;
+ if (!forcedWhereFilter && filter instanceof ConditionFilter) {
+ final ConditionFilter conditionFilter = (ConditionFilter) filter;
+ if (conditionFilter.hasVirtualRowVariables()) {
+ throw new UnsupportedOperationException(
+ "UpdateBy CountWhere operator does not support refreshing filters");
+ }
+ try {
+ countFilter = new CountFilter(conditionFilter.getFilter(inputTable, RowSetFactory.empty()),
+ filterInputIndices.get(fi));
+ } catch (final Exception e) {
+ throw new IllegalArgumentException(
+ "Error creating condition filter in UpdateBy CountWhere Operator", e);
+ }
+ } else if (!forcedWhereFilter && filter instanceof ExposesChunkFilter
+ && ((ExposesChunkFilter) filter).chunkFilter().isPresent()) {
+ final Optional chunkFilter = ((ExposesChunkFilter) filter).chunkFilter();
+ countFilter = new CountFilter(chunkFilter.get(), filterInputIndices.get(fi));
+ } else {
+ try (final SafeCloseable ignored = filter.beginOperation(inputTable)) {
+ countFilter = new CountFilter(filter, filterInputIndices.get(fi));
+ }
+ forcedWhereFilter = true;
+ }
+ filterList.add(countFilter);
+ }
+
+ return filterList.toArray(CountFilter[]::new);
+ }
+ }
+
+ private class Context extends BaseLongUpdateByOperator.Context {
+ /**
+ * The chunk sources that populate the chunkSourceTable for this context.
+ */
+ private final ChunkColumnSource>[] chunkColumnSources;
+ /**
+ * The chunk sources that help populate the original-typed chunks for this context.
+ */
+ private final ColumnSource>[] contextOriginalColumnSources;
+ /**
+ * Fill contexts to help convert back to original-typed chunks.
+ */
+ private final ChunkSource.FillContext[] originalFillContexts;
+ /**
+ * Chunks to store original-typed values for condition and chunk filters.
+ */
+ private final WritableChunk super Values>[] originalValueChunks;
+
+ /**
+ * A table composed of chunk sources for generic where filters for this context.
+ */
+ private final Table chunkSourceTable;
+ /**
+ * Contains the results of the filters as a boolean chunk, where true indicates a row passed all filters.
+ */
+ private final WritableBooleanChunk resultsChunk;
+ /**
+ * The ring buffer representing the filter output for the values currently in the window.
+ */
+ private final ByteRingBuffer buffer;
+ /**
+ * The raw filters to apply to the data, these may diverge from the operator filters.
+ */
+ private final CountFilter[] contextFilters;
+ /**
+ * The chunk data from the recorders to be used as input to the filters.
+ */
+ private final Chunk extends Values>[][] filterChunks;
+ /**
+ * ConditionFilters need a context to store intermediate results.
+ */
+ final ConditionFilter.FilterKernel.Context[] conditionFilterContexts;
+
+ /**
+ * Store the SafeCloseable items returned by the WhereFilter#beginOperation() calls for clean up when the
+ * context is released.
+ */
+ final SafeCloseable[] filterOperationContexts;
+
+ /**
+ * A list of all items that need to be closed when the context is released.
+ */
+ final SafeCloseableList closeableList;
+
+ @SuppressWarnings("unused")
+ private Context(final int affectedChunkSize, final int influencerChunkSize) {
+ super(affectedChunkSize);
+
+ closeableList = new SafeCloseableList();
+
+ if (originalChunksRequired) {
+ originalFillContexts = new ChunkSource.FillContext[inputColumnNames.length];
+ // noinspection unchecked
+ originalValueChunks = new WritableChunk[inputColumnNames.length];
+ } else {
+ originalFillContexts = null;
+ originalValueChunks = null;
+ }
+
+ // Create a new chunk source table for this context
+ if (chunkSourceTableRequired) {
+ contextOriginalColumnSources = new ColumnSource>[inputColumnNames.length];
+ final Map> columnSourceMap = new LinkedHashMap<>();
+ chunkColumnSources = new ChunkColumnSource>[inputColumnNames.length];
+ for (int i = 0; i < inputColumnNames.length; i++) {
+ final ColumnSource> inputSource = reinterpretedSources[i];
+ chunkColumnSources[i] = ChunkColumnSource.make(inputSource.getChunkType(), inputSource.getType(),
+ inputSource.getComponentType());
+ if (originalSources[i] != null) {
+ // This column needs to be interpreted back to the original type.
+ final ColumnSource> toOriginal =
+ ReinterpretUtils.convertToOriginalType(originalSources[i], chunkColumnSources[i]);
+ // Put the re-reinterpreted column source in the map
+ columnSourceMap.put(inputColumnNames[i], toOriginal);
+ // Create chunk and fill context for original-typed converted values
+ originalFillContexts[i] = toOriginal.makeFillContext(influencerChunkSize);
+ closeableList.add(originalFillContexts[i]);
+ originalValueChunks[i] = toOriginal.getChunkType().makeWritableChunk(influencerChunkSize);
+ closeableList.add(originalValueChunks[i]);
+ contextOriginalColumnSources[i] = toOriginal;
+ } else {
+ columnSourceMap.put(inputColumnNames[i], chunkColumnSources[i]);
+ }
+ }
+ chunkSourceTable = new QueryTable(RowSetFactory.empty().toTracking(), columnSourceMap);
+ } else {
+ chunkColumnSources = null;
+ chunkSourceTable = null;
+ contextOriginalColumnSources = null;
+ }
+
+ // Store the operator's chunk and condition filters, but create new WhereFilters. This is required because
+ // the operator WhereFilters were initialized against a dummy table.
+ contextFilters = new CountFilter[filters.length];
+ filterOperationContexts = new SafeCloseable[filters.length];
+
+ for (int fi = 0; fi < filters.length; fi++) {
+ final CountFilter filter = filters[fi];
+ if (filter.whereFilter != null) {
+ Require.neqNull(chunkSourceTable, "chunkSourceTable");
+ final WhereFilter copiedFilter = filter.whereFilter.copy();
+
+ // Initialize the copied filter against this context chunk source table.
+ copiedFilter.init(chunkSourceTable.getDefinition());
+
+ // noinspection resource
+ filterOperationContexts[fi] = copiedFilter.beginOperation(chunkSourceTable);
+ closeableList.add(filterOperationContexts[fi]);
+
+ // Use this WhereFilter in chunk processing
+ contextFilters[fi] = new CountFilter(copiedFilter, filter.inputColumnIndices);
+ } else {
+ contextFilters[fi] = filter;
+ }
+ }
+
+ conditionFilterContexts = new ConditionFilter.FilterKernel.Context[filters.length];
+ for (int ii = 0; ii < filters.length; ii++) {
+ if (filters[ii].conditionFilter != null) {
+ conditionFilterContexts[ii] = filters[ii].conditionFilter.getContext(influencerChunkSize);
+ closeableList.add(conditionFilterContexts[ii]);
+ }
+ }
+
+ // noinspection unchecked
+ filterChunks = new Chunk[filters.length][];
+ for (int ii = 0; ii < filters.length; ii++) {
+ final CountFilter filter = filters[ii];
+ // noinspection unchecked
+ filterChunks[ii] = new Chunk[filter.inputColumnIndices.length];
+ }
+
+ resultsChunk = WritableBooleanChunk.makeWritableChunk(influencerChunkSize);
+ closeableList.add(resultsChunk);
+
+ buffer = new ByteRingBuffer(BUFFER_INITIAL_CAPACITY, true);
+
+ curVal = 0;
+ }
+
+ @Override
+ public void reset() {
+ buffer.clear();
+ curVal = 0;
+ }
+
+ @Override
+ public void close() {
+ super.close();
+ closeableList.close();
+ }
+
+ /**
+ * Assign the input chunks to the correct filters, maybe reinterpreting the chunks to the original type.
+ */
+ private void assignInputChunksToFilters(
+ @NotNull Chunk extends Values>[] valueChunks,
+ final int influencerChunkSize) {
+
+ try (final RowSet influencerRs = originalChunksRequired ? RowSetFactory.flat(influencerChunkSize) : null) {
+ // Update the chunk source table chunks if needed.
+ if (chunkSourceTableRequired) {
+ for (int i = 0; i < inputColumnNames.length; i++) {
+ chunkColumnSources[i].clear();
+ chunkColumnSources[i].addChunk((WritableChunk extends Values>) valueChunks[i]);
+ }
+ }
+
+ // Assign the filter input chunks from the value chunks.
+ for (int fi = 0; fi < filters.length; fi++) {
+ final CountFilter filter = filters[fi];
+
+ for (int fci = 0; fci < filter.inputColumnIndices.length; fci++) {
+ final int inputIndex = filter.inputColumnIndices[fci];
+ if (originalChunksRequired && contextOriginalColumnSources[inputIndex] != null) {
+ // Filling from contextOriginalColumnSources[] will produce original typed version of the
+ // input chunk data since we seeded the underlying ChunkColumnSource with the input chunk.
+ contextOriginalColumnSources[inputIndex].fillChunk(
+ originalFillContexts[inputIndex],
+ originalValueChunks[inputIndex],
+ influencerRs);
+ // noinspection unchecked
+ filterChunks[fi][fci] = (Chunk extends Values>) originalValueChunks[inputIndex];
+ } else {
+ filterChunks[fi][fci] = valueChunks[inputIndex];
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Do the work of applying the filters against the input data and assigning true to the result chunk where all
+ * filters pass, false otherwise.
+ */
+ private void applyFilters(final int chunkSize) {
+ // Use the filters to populate a boolean buffer with the filter results.
+ boolean initialized = false;
+ WritableRowSet remainingRows = null;
+ final RowSet flatRowSet = RowSetFactory.flat(chunkSize);
+
+ // We must apply the filters in the order they were given.
+ for (int fi = 0; fi < contextFilters.length; fi++) {
+ final CountFilter filter = contextFilters[fi];
+ final Chunk extends Values>[] valueChunks = filterChunks[fi];
+ final ConditionFilter.FilterKernel.Context conditionalFilterContext = conditionFilterContexts[fi];
+
+ if (filter.chunkFilter != null) {
+ if (!initialized) {
+ // Chunk filters only have one input.
+ filter.chunkFilter.filter(valueChunks[0], resultsChunk);
+ initialized = true;
+ } else {
+ filter.chunkFilter.filterAnd(valueChunks[0], resultsChunk);
+ }
+ } else if (filter.conditionFilter != null) {
+ if (!initialized) {
+ filter.conditionFilter.filter(conditionalFilterContext, valueChunks, chunkSize, resultsChunk);
+ initialized = true;
+ } else {
+ filter.conditionFilter.filterAnd(conditionalFilterContext, valueChunks, chunkSize,
+ resultsChunk);
+ }
+ } else {
+ if (remainingRows == null) {
+ // This is the first WhereFilter to run, initialize the remainingRows RowSet
+ remainingRows = initialized
+ ? buildFromBooleanChunk(resultsChunk, chunkSize)
+ : RowSetFactory.flat(chunkSize);
+ }
+ try (final RowSet ignored = remainingRows) {
+ remainingRows = filter.whereFilter.filter(remainingRows, flatRowSet, chunkSourceTable, false);
+ }
+ initialized = true;
+ }
+ }
+
+ try (final RowSet ignored = remainingRows; final RowSet ignored2 = flatRowSet) {
+ if (remainingRows != null) {
+ // WhereFilters were used, so gather the info from remainingRows
+ resultsChunk.fillWithValue(0, chunkSize, false);
+ remainingRows.forAllRowKeyRanges(
+ (start, end) -> resultsChunk.fillWithValue((int) start, (int) end - (int) start + 1,
+ true));
+ }
+ }
+ }
+
+ @Override
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
+ @NotNull final Chunk extends Values>[] influencerValueChunkArr,
+ @Nullable final LongChunk affectedPosChunk,
+ @Nullable final LongChunk influencerPosChunk,
+ @NotNull final IntChunk extends Values> pushChunk,
+ @NotNull final IntChunk extends Values> popChunk,
+ final int affectedCount,
+ final int influencerCount) {
+
+ assignInputChunksToFilters(influencerValueChunkArr, influencerCount);
+ setPosChunks(affectedPosChunk, influencerPosChunk);
+
+ applyFilters(influencerCount);
+
+ int pushIndex = 0;
+
+ // chunk processing
+ for (int ii = 0; ii < affectedCount; ii++) {
+ final int pushCount = pushChunk.get(ii);
+ final int popCount = popChunk.get(ii);
+
+ if (pushCount == NULL_INT) {
+ writeNullToOutputChunk(ii);
+ continue;
+ }
+
+ // pop for this row
+ if (popCount > 0) {
+ pop(popCount);
+ }
+
+ // push for this row
+ if (pushCount > 0) {
+ push(pushIndex, pushCount);
+ pushIndex += pushCount;
+ }
+
+ // write the results to the output chunk
+ writeToOutputChunk(ii);
+ }
+
+ // chunk output to column
+ writeToOutputColumn(inputKeys);
+ }
+
+ @Override
+ public void accumulateCumulative(
+ @NotNull final RowSequence inputKeys,
+ @NotNull final Chunk extends Values>[] valueChunkArr,
+ @Nullable final LongChunk extends Values> tsChunk,
+ final int len) {
+
+ assignInputChunksToFilters(valueChunkArr, len);
+ applyFilters(len);
+
+ // chunk processing
+ for (int ii = 0; ii < len; ii++) {
+ push(ii, 1);
+ writeToOutputChunk(ii);
+ }
+
+ // chunk output to column
+ writeToOutputColumn(inputKeys);
+ }
+
+ @Override
+ protected void push(int pos, int count) {
+ // Push from the pre-computed results chunk into the buffer.
+ buffer.ensureRemaining(count);
+
+ for (int ii = 0; ii < count; ii++) {
+ final boolean val = resultsChunk.get(pos + ii);
+ if (val) {
+ curVal++;
+ buffer.add((byte) 1);
+ } else {
+ buffer.add((byte) 0);
+ }
+ }
+ }
+
+ @Override
+ protected void pop(int count) {
+ Assert.geq(buffer.size(), "buffer.size()", count);
+
+ for (int ii = 0; ii < count; ii++) {
+ final byte val = buffer.removeUnsafe();
+
+ if (val == 1) {
+ curVal--;
+ }
+ }
+ }
+ }
+
+ /**
+ * Build a RowSet from a boolean chunk, including only the indices that are true.
+ */
+ private static WritableRowSet buildFromBooleanChunk(final BooleanChunk values, final int chunkSize) {
+ final RowSetBuilderSequential builder = RowSetFactory.builderSequential();
+ for (int ii = 0; ii < chunkSize; ii++) {
+ if (values.get(ii)) {
+ builder.appendKey(ii);
+ }
+ }
+ return builder.build();
+ }
+
+ /**
+ * Create a new CountWhereOperator for rolling / windowed operations.
+ *
+ * @param pair Contains the output column name as a MatchPair
+ * @param affectingColumns The names of the columns that when changed would affect this formula output
+ * @param timestampColumnName The name of the column containing timestamps for time-based calculations (or null when
+ * not time-based)
+ * @param reverseWindowScaleUnits The size of the reverse window in ticks (or nanoseconds when time-based)
+ * @param forwardWindowScaleUnits The size of the forward window in ticks (or nanoseconds when time-based)
+ * @param filters the filters to apply to the input columns
+ * @param inputColumnNames The names of the key columns to be used as inputs
+ * @param originalSources Representative original sources for the input columns; stores type information and helps
+ * convert input data chunks but does not hold any actual data
+ * @param reinterpretedSources Representative reinterpreted sources for input columns; stores type information but
+ * does not hold any actual data
+ * @param chunkSourceTableRequired Whether we need to create and maintain a chunk source table
+ * @param originalChunksRequired Whether we need to produce original-typed chunks for condition and chunk filters
+ */
+ public CountWhereOperator(
+ @NotNull final MatchPair pair,
+ @NotNull final String[] affectingColumns,
+ @Nullable final String timestampColumnName,
+ final long reverseWindowScaleUnits,
+ final long forwardWindowScaleUnits,
+ final CountFilter[] filters,
+ final String[] inputColumnNames,
+ final ColumnSource>[] originalSources,
+ final ColumnSource>[] reinterpretedSources,
+ final boolean chunkSourceTableRequired,
+ final boolean originalChunksRequired) {
+ super(pair, affectingColumns, timestampColumnName, reverseWindowScaleUnits, forwardWindowScaleUnits, true);
+ this.filters = filters;
+ this.inputColumnNames = inputColumnNames;
+ this.originalSources = originalSources;
+ this.reinterpretedSources = reinterpretedSources;
+ this.chunkSourceTableRequired = chunkSourceTableRequired;
+ this.originalChunksRequired = originalChunksRequired;
+ }
+
+ /**
+ * Create a new CountWhereOperator for cumulative operations.
+ *
+ * @param pair Contains the output column name as a MatchPair
+ * @param filters the filters to apply to the input columns
+ * @param originalSources Representative original sources for the input columns; stores type information and helps
+ * convert input data chunks but does not hold any actual data
+ * @param reinterpretedSources Representative reinterpreted sources for input columns; stores type information but
+ * does not hold any actual data
+ * @param chunkSourceTableRequired Whether we need to create and maintain a chunk source table
+ * @param originalChunksRequired Whether we need to produce original-typed chunks for condition and chunk filters
+ */
+ public CountWhereOperator(
+ @NotNull final MatchPair pair,
+ final CountFilter[] filters,
+ final String[] inputColumnNames,
+ final ColumnSource>[] originalSources,
+ final ColumnSource>[] reinterpretedSources,
+ final boolean chunkSourceTableRequired,
+ final boolean originalChunksRequired) {
+ super(pair, inputColumnNames, null, 0, 0, false);
+ this.filters = filters;
+ this.inputColumnNames = inputColumnNames;
+ this.originalSources = originalSources;
+ this.reinterpretedSources = reinterpretedSources;
+ this.chunkSourceTableRequired = chunkSourceTableRequired;
+ this.originalChunksRequired = originalChunksRequired;
+ }
+
+ @Override
+ public UpdateByOperator copy() {
+ if (!isWindowed) {
+ return new CountWhereOperator(
+ pair,
+ filters,
+ inputColumnNames,
+ originalSources,
+ reinterpretedSources,
+ chunkSourceTableRequired,
+ originalChunksRequired);
+ }
+ return new CountWhereOperator(
+ pair,
+ affectingColumns,
+ timestampColumnName,
+ reverseWindowScaleUnits,
+ forwardWindowScaleUnits,
+ filters,
+ inputColumnNames,
+ originalSources,
+ reinterpretedSources,
+ chunkSourceTableRequired,
+ originalChunksRequired);
+ }
+
+ /**
+ * This must be overridden to provide the input column chunks that will be needed for computation.
+ */
+ @Override
+ @NotNull
+ protected String[] getInputColumnNames() {
+ return ArrayUtils.addAll(inputColumnNames);
+ }
+
+ @Override
+ public UpdateByOperator.@NotNull Context makeUpdateContext(int affectedChunkSize, int influencerChunkSize) {
+ return new Context(affectedChunkSize, influencerChunkSize);
+ }
+}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java
index 451b285beb1..738b746cb69 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseByteUpdateByOperator.java
@@ -73,13 +73,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -87,7 +89,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -123,7 +125,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_BYTE);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java
index 7a6c44999ba..fa229ab6ca3 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseCharUpdateByOperator.java
@@ -63,13 +63,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -77,7 +79,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -113,7 +115,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_CHAR);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java
index ba2585be16b..5f4ff7be489 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseDoubleUpdateByOperator.java
@@ -67,13 +67,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -81,7 +83,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -117,7 +119,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_DOUBLE);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java
index 5dff3066b43..35364d6a241 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseFloatUpdateByOperator.java
@@ -67,13 +67,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -81,7 +83,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -117,7 +119,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_FLOAT);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java
index 112284e5a48..042580dacf6 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseIntUpdateByOperator.java
@@ -67,13 +67,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -81,7 +83,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -117,7 +119,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_INT);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java
index 52410ab4d0b..19bf9de5c4e 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseLongUpdateByOperator.java
@@ -67,13 +67,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -81,7 +83,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -117,7 +119,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_LONG);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java
index 8d06f8195bf..cd0c7619c2e 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseObjectUpdateByOperator.java
@@ -70,13 +70,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -84,7 +86,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -120,7 +122,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, null);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java
index 0a5107d4ee0..3cebdb795b2 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/internal/BaseShortUpdateByOperator.java
@@ -67,13 +67,15 @@ public void accumulateCumulative(@NotNull final RowSequence inputKeys,
}
@Override
- public void accumulateRolling(@NotNull final RowSequence inputKeys,
+ public void accumulateRolling(
+ @NotNull final RowSequence inputKeys,
@NotNull final Chunk extends Values>[] influencerValueChunkArr,
@Nullable final LongChunk affectedPosChunk,
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -81,7 +83,7 @@ public void accumulateRolling(@NotNull final RowSequence inputKeys,
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
@@ -117,7 +119,7 @@ public void writeToOutputChunk(final int outIdx) {
outputValues.set(outIdx, curVal);
}
- void writeNullToOutputChunk(final int outIdx) {
+ protected void writeNullToOutputChunk(final int outIdx) {
outputValues.set(outIdx, NULL_SHORT);
}
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/BooleanRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/BooleanRollingFormulaOperator.java
index 7144e591683..fb02cd1b1d6 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/BooleanRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/BooleanRollingFormulaOperator.java
@@ -91,7 +91,7 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount, int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -99,7 +99,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ByteRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ByteRollingFormulaOperator.java
index 477d949f647..f7e308e8033 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ByteRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ByteRollingFormulaOperator.java
@@ -94,7 +94,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -102,7 +103,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/CharRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/CharRollingFormulaOperator.java
index 1882f881aad..ab6d29be02d 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/CharRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/CharRollingFormulaOperator.java
@@ -90,7 +90,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -98,7 +99,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/DoubleRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/DoubleRollingFormulaOperator.java
index 2a4f4026415..a3dcc02b742 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/DoubleRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/DoubleRollingFormulaOperator.java
@@ -94,7 +94,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -102,7 +103,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/FloatRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/FloatRollingFormulaOperator.java
index 398685f8da9..629107f712a 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/FloatRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/FloatRollingFormulaOperator.java
@@ -94,7 +94,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -102,7 +103,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/IntRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/IntRollingFormulaOperator.java
index e3b432dde19..1787756f28b 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/IntRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/IntRollingFormulaOperator.java
@@ -94,7 +94,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -102,7 +103,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/LongRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/LongRollingFormulaOperator.java
index dc2051b2701..50991b8636b 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/LongRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/LongRollingFormulaOperator.java
@@ -94,7 +94,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -102,7 +103,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ObjectRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ObjectRollingFormulaOperator.java
index c344a150ca5..39526c26232 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ObjectRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ObjectRollingFormulaOperator.java
@@ -87,7 +87,7 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount, int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -95,7 +95,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ShortRollingFormulaOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ShortRollingFormulaOperator.java
index 1ab9962c011..ba2c0a5af56 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ShortRollingFormulaOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformula/ShortRollingFormulaOperator.java
@@ -94,7 +94,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -102,7 +103,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformulamulticolumn/RollingFormulaMultiColumnOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformulamulticolumn/RollingFormulaMultiColumnOperator.java
index ca1effa60e3..fe6aacb196f 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformulamulticolumn/RollingFormulaMultiColumnOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollingformulamulticolumn/RollingFormulaMultiColumnOperator.java
@@ -154,7 +154,8 @@ public void accumulateRolling(
@Nullable final LongChunk influencerPosChunk,
@NotNull final IntChunk extends Values> pushChunk,
@NotNull final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
setValueChunks(influencerValueChunkArr);
setPosChunks(affectedPosChunk, influencerPosChunk);
@@ -162,7 +163,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
diff --git a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollinggroup/RollingGroupOperator.java b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollinggroup/RollingGroupOperator.java
index f6ec36769a9..999ddaec794 100644
--- a/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollinggroup/RollingGroupOperator.java
+++ b/engine/table/src/main/java/io/deephaven/engine/table/impl/updateby/rollinggroup/RollingGroupOperator.java
@@ -120,7 +120,8 @@ public void accumulateRolling(
final LongChunk influencerPosChunk,
final IntChunk extends Values> pushChunk,
final IntChunk extends Values> popChunk,
- final int len) {
+ final int affectedCount,
+ final int influencerCount) {
if (timestampColumnName == null) {
// The only work for ticks operators is to update the groupRowSetSource
@@ -133,7 +134,7 @@ public void accumulateRolling(
int pushIndex = 0;
// chunk processing
- for (int ii = 0; ii < len; ii++) {
+ for (int ii = 0; ii < affectedCount; ii++) {
final int pushCount = pushChunk.get(ii);
final int popCount = popChunk.get(ii);
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 9ecb8e14e42..32e252f8e13 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
@@ -341,6 +341,21 @@ public void testAggCountWhere() {
assertEquals(6L, counts.get(0));
counts = ColumnVectors.ofLong(doubleCounted, "filter15");
assertEquals(6L, counts.get(0));
+
+ // Get a static set table for use in dynamic where filters (contains 0-3)
+ final QueryTable setTable = (QueryTable) TableTools.newTable(col("sym", 1, 2, 3));
+
+ doubleCounted = table.aggBy(
+ List.of(AggCountWhere("normal",
+ new DynamicWhereFilter(setTable, true, new MatchPair("B", "sym"))),
+ AggCountWhere("invert",
+ new DynamicWhereFilter(setTable, false, new MatchPair("B", "sym")))));
+ show(doubleCounted);
+ assertEquals(1, doubleCounted.size());
+ counts = ColumnVectors.ofLong(doubleCounted, "normal");
+ assertEquals(3L, counts.get(0));
+ counts = ColumnVectors.ofLong(doubleCounted, "invert");
+ assertEquals(7L, counts.get(0));
}
@Test
diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumCountWhere.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumCountWhere.java
new file mode 100644
index 00000000000..37a8eca7fdb
--- /dev/null
+++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestCumCountWhere.java
@@ -0,0 +1,703 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.engine.table.impl.updateby;
+
+import io.deephaven.api.filter.Filter;
+import io.deephaven.api.updateby.UpdateByOperation;
+import io.deephaven.base.Predicate;
+import io.deephaven.base.verify.Assert;
+import io.deephaven.engine.context.ExecutionContext;
+import io.deephaven.engine.primitive.iterator.CloseableIterator;
+import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfInt;
+import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfLong;
+import io.deephaven.engine.table.PartitionedTable;
+import io.deephaven.engine.table.Table;
+import io.deephaven.engine.table.impl.MatchPair;
+import io.deephaven.engine.table.impl.QueryTable;
+import io.deephaven.engine.table.impl.select.DynamicWhereFilter;
+import io.deephaven.engine.testutil.ControlledUpdateGraph;
+import io.deephaven.engine.testutil.EvalNugget;
+import io.deephaven.engine.testutil.TstUtils;
+import io.deephaven.engine.testutil.generator.CharGenerator;
+import io.deephaven.engine.testutil.generator.SortedInstantGenerator;
+import io.deephaven.engine.testutil.generator.TestDataGenerator;
+import io.deephaven.engine.util.TableTools;
+import io.deephaven.test.types.OutOfBandTest;
+import io.deephaven.time.DateTimeUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.time.Instant;
+import java.util.Random;
+
+import static io.deephaven.engine.testutil.GenerateTableUpdates.generateAppends;
+import static io.deephaven.engine.testutil.TstUtils.assertTableEquals;
+import static io.deephaven.engine.testutil.testcase.RefreshingTableTestCase.simulateShiftAwareStep;
+
+@Category(OutOfBandTest.class)
+public class TestCumCountWhere extends BaseUpdateByTest {
+ final int STATIC_TABLE_SIZE = 10_000;
+ final int DYNAMIC_TABLE_SIZE = 1_000;
+ final int DYNAMIC_UPDATE_SIZE = 100;
+ final int DYNAMIC_UPDATE_STEPS = 20;
+
+ // region Object Helper functions
+ private static class TestHelper {
+ private static void assertWhereInt(
+ final CloseablePrimitiveIteratorOfLong actualIt,
+ final CloseablePrimitiveIteratorOfInt valueIt,
+ final Predicate.Int predicate) {
+ if (!actualIt.hasNext()) {
+ return;
+ }
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ if (predicate.call(valueIt.nextInt())) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ private static void assertWhereObject(
+ final CloseablePrimitiveIteratorOfLong actualIt,
+ final CloseableIterator valueIt,
+ final Predicate.Unary predicate) {
+ if (!actualIt.hasNext()) {
+ return;
+ }
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ if (predicate.call(valueIt.next())) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+ }
+ // endregion Object Helper functions
+
+ // region Zero Key Tests
+
+ @Test
+ public void testStaticZeroKey() {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, false, false, 0xFFFABBBC,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)}).t;
+
+ Table actual;
+
+ // Test simple ChunkFilter, int > 50
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val > 50);
+ }
+
+ // Test simple ChunkFilter, int <= 50
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val <= 50);
+ }
+
+ // Test simple conditional filter, true
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "true"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> true);
+ }
+
+ // Test simple conditional filter, false
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "false"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> false);
+ }
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", "intCol > 10 && intCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val > 10 && val <= 50);
+ }
+
+ // Test on String column (representing all Object)
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count",
+ "Sym != null && Sym.startsWith(`A`)"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = t.columnIterator("Sym")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt, val -> val != null && val.startsWith("A"));
+ }
+
+ // Test OR filter (processed as a WhereFilter)
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count",
+ Filter.or(Filter.from("intCol < 25", "intCol > 75"))));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val < 25 || val > 75);
+ }
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = t.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = t.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if ((val1 < 25 || val1 > 75) && (val2 < 25 || val2 > 75)) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", "intCol > 10", "longCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = t.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = t.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if (val1 > 10 && val2 <= 50) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", "intCol > 10 || longCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = t.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = t.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if (val1 > 10 || val2 <= 50) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", filter));
+
+ // The set table contains 0-29, so we can compare against an equivalent filter.
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val >= 0 && val <= 29);
+ }
+
+ // Test Boolean column
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "!isNull(boolCol) && boolCol"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = t.columnIterator("boolCol")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt, val -> val != null && val);
+ }
+
+ // Test Instant column
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = t.columnIterator("ts")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt,
+ val -> val.isAfter(DateTimeUtils.parseInstant("2022-03-09T12:00:00.000 NY")));
+ }
+ }
+
+ @Test
+ public void testStaticZeroKeyAllNulls() {
+ final QueryTable t = createTestTableAllNull(STATIC_TABLE_SIZE, true, false, false, 0xFFFABBBC,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)}).t;
+
+ Table actual;
+
+ // Test simple ChunkFilter, int > 50
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val > 50);
+ }
+
+ // Test simple ChunkFilter, int <= 50
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val <= 50);
+ }
+
+ // Test simple conditional filter, true
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "true"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> true);
+ }
+
+ // Test simple conditional filter, false
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "false"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> false);
+ }
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", "intCol > 10 && intCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val > 10 && val <= 50);
+ }
+
+ // Test on String column (representing all Object)
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count",
+ "Sym != null && Sym.startsWith(`A`)"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = t.columnIterator("Sym")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt, val -> val != null && val.startsWith("A"));
+ }
+
+ // Test OR filter (processed as a WhereFilter)
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count",
+ Filter.or(Filter.from("intCol < 25", "intCol > 75"))));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val < 25 || val > 75);
+ }
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = t.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = t.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if ((val1 < 25 || val1 > 75) && (val2 < 25 || val2 > 75)) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", "intCol > 10", "longCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = t.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = t.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if (val1 > 10 && val2 <= 50) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", "intCol > 10 || longCol <= 50"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = t.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = t.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if (val1 > 10 || val2 <= 50) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", filter));
+
+ // The set table contains 0-29, so we can compare against an equivalent filter.
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = t.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val >= 0 && val <= 29);
+ }
+
+ // Test Boolean column
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count", "!isNull(boolCol) && boolCol"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = t.columnIterator("boolCol")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt, val -> val != null && val);
+ }
+
+ // Test Instant column
+ actual = t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)"));
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = t.columnIterator("ts")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt,
+ val -> val.isAfter(DateTimeUtils.parseInstant("2022-03-09T12:00:00.000 NY")));
+ }
+ }
+
+ // endregion
+
+ // region Bucketed Tests
+
+ @Test
+ public void testStaticBucketed() {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, false, false, 0xFFFABBBC,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)}).t;
+
+ final PartitionedTable preOp = t.partitionBy("Sym");
+
+ PartitionedTable postOp;
+
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val > 50);
+ }
+ return source;
+ });
+
+ // Test simple ChunkFilter, int <= 50
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol <= 50"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val <= 50);
+ }
+ return source;
+ });
+
+ // Test simple conditional filter, true
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "true"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> true);
+ }
+ return source;
+ });
+
+ // Test simple conditional filter, false
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "false"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> false);
+ }
+ return source;
+ });
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 10 && intCol <= 50"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val > 10 && val <= 50);
+ }
+ return source;
+ });
+
+ // Test on String column (representing all Object)
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "Sym != null && Sym.startsWith(`A`)"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = source.columnIterator("Sym")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt, val -> val != null && val.startsWith("A"));
+ }
+ return source;
+ });
+
+ // Test OR filter (processed as a WhereFilter)
+ postOp = t.updateBy(
+ UpdateByOperation.CumCountWhere("count", Filter.or(Filter.from("intCol < 25", "intCol > 75"))), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val < 25 || val > 75);
+ }
+ return source;
+ });
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ postOp = t
+ .updateBy(UpdateByOperation.CumCountWhere("count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))),
+ "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = source.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = source.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if ((val1 < 25 || val1 > 75) && (val2 < 25 || val2 > 75)) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+ return source;
+ });
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 10", "longCol <= 50"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = source.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = source.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if (val1 > 10 && val2 <= 50) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+ return source;
+ });
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 10 || longCol <= 50"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt1 = source.integerColumnIterator("intCol");
+ final CloseablePrimitiveIteratorOfLong expectedIt2 = source.longColumnIterator("longCol")) {
+
+ long count = 0;
+ while (actualIt.hasNext()) {
+ final int val1 = expectedIt1.nextInt();
+ final long val2 = expectedIt2.nextLong();
+
+ if (val1 > 10 || val2 <= 50) {
+ count++;
+ }
+ Assert.eq(actualIt.nextLong(), "values match", count);
+ }
+ }
+ return source;
+ });
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", filter), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseablePrimitiveIteratorOfInt expectedIt = source.integerColumnIterator("intCol")) {
+ TestHelper.assertWhereInt(actualIt, expectedIt, val -> val >= 0 && val <= 29);
+ }
+ return source;
+ });
+
+
+ // Test Boolean column
+ postOp = t.updateBy(UpdateByOperation.CumCountWhere("count", "!isNull(boolCol) && boolCol"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = source.columnIterator("boolCol")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt, val -> val != null && val);
+ }
+ return source;
+ });
+
+ // Test Instant column
+ postOp = t
+ .updateBy(UpdateByOperation.CumCountWhere("count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)"), "Sym")
+ .partitionBy("Sym");
+ preOp.partitionedTransform(postOp, (source, actual) -> {
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedIt = source.columnIterator("ts")) {
+ TestHelper.assertWhereObject(actualIt, expectedIt,
+ val -> val.isAfter(DateTimeUtils.parseInstant("2022-03-09T12:00:00.000 NY")));
+ }
+ return source;
+ });
+ }
+
+ // endregion
+
+ // region Live Tests
+
+ @Test
+ public void testZeroKeyAppendOnly() {
+ doTestTicking(false, true);
+ }
+
+ @Test
+ public void testBucketedAppendOnly() {
+ doTestTicking(true, true);
+ }
+
+ @Test
+ public void testZeroKeyGeneral() {
+ doTestTicking(false, false);
+ }
+
+ @Test
+ public void testBucketedGeneral() {
+ doTestTicking(true, false);
+ }
+
+ private void doTestTicking(boolean bucketed, boolean appendOnly) {
+ final int seed = 0xB177B177;
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, bucketed, false, true, seed,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)});
+
+ final QueryTable t = result.t;
+
+ if (appendOnly) {
+ t.setAttribute(Table.ADD_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
+ }
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"),
+ "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50",
+ "intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50",
+ "intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "intCol > 50 && intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "intCol > 50 && intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count", "false"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count", "false"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50",
+ "longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50",
+ "longCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "intCol > 50 && longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "intCol > 50 && longCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "boolCol != null && boolCol"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "boolCol != null && boolCol"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)"), "Sym")
+ : t.updateBy(UpdateByOperation.CumCountWhere("count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)")))
+ };
+
+ final Random random = new Random(seed);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ if (appendOnly) {
+ ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle(
+ () -> generateAppends(DYNAMIC_UPDATE_SIZE, random, t, result.infos));
+ TstUtils.validate("Table", nuggets);
+ } else {
+ simulateShiftAwareStep(DYNAMIC_UPDATE_SIZE, random, t, result.infos, nuggets);
+ }
+ }
+ }
+ // endregion
+
+ @Test
+ public void testProxy() {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, false, false, 0x31313131).t;
+
+ Table actual;
+ Table expected;
+
+ // Compare the merged proxy table to the bucketed version (need to sort by symbol to get alignment).
+ PartitionedTable pt = t.partitionBy("Sym");
+ actual = pt.proxy()
+ .updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"))
+ .target().merge().sort("Sym");
+ expected = t
+ .updateBy(UpdateByOperation.CumCountWhere("count", "intCol > 50"), "Sym")
+ .sort("Sym");
+ assertTableEquals(expected, actual);
+
+ actual = pt.proxy()
+ .updateBy(UpdateByOperation.CumCountWhere("count", "intCol <= 50"))
+ .target().merge().sort("Sym");
+ expected = t
+ .updateBy(UpdateByOperation.CumCountWhere("count", "intCol <= 50"), "Sym")
+ .sort("Sym");
+ assertTableEquals(expected, actual);
+ }
+}
diff --git a/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCountWhere.java b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCountWhere.java
new file mode 100644
index 00000000000..9cf813253f6
--- /dev/null
+++ b/engine/table/src/test/java/io/deephaven/engine/table/impl/updateby/TestRollingCountWhere.java
@@ -0,0 +1,1836 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.engine.table.impl.updateby;
+
+import io.deephaven.api.filter.Filter;
+import io.deephaven.api.updateby.UpdateByControl;
+import io.deephaven.api.updateby.UpdateByOperation;
+import io.deephaven.base.Predicate;
+import io.deephaven.base.verify.Assert;
+import io.deephaven.engine.context.ExecutionContext;
+import io.deephaven.engine.primitive.iterator.CloseableIterator;
+import io.deephaven.engine.primitive.iterator.CloseablePrimitiveIteratorOfLong;
+import io.deephaven.engine.table.PartitionedTable;
+import io.deephaven.engine.table.Table;
+import io.deephaven.engine.table.impl.MatchPair;
+import io.deephaven.engine.table.impl.QueryTable;
+import io.deephaven.engine.table.impl.select.DynamicWhereFilter;
+import io.deephaven.engine.testutil.ControlledUpdateGraph;
+import io.deephaven.engine.testutil.EvalNugget;
+import io.deephaven.engine.testutil.GenerateTableUpdates;
+import io.deephaven.engine.testutil.TstUtils;
+import io.deephaven.engine.testutil.generator.CharGenerator;
+import io.deephaven.engine.testutil.generator.SortedInstantGenerator;
+import io.deephaven.engine.testutil.generator.TestDataGenerator;
+import io.deephaven.engine.util.TableTools;
+import io.deephaven.test.types.OutOfBandTest;
+import io.deephaven.time.DateTimeUtils;
+import io.deephaven.vector.IntVector;
+import io.deephaven.vector.LongVector;
+import io.deephaven.vector.ObjectVector;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Random;
+
+import static io.deephaven.engine.testutil.GenerateTableUpdates.generateAppends;
+import static io.deephaven.engine.testutil.TstUtils.assertTableEquals;
+import static io.deephaven.engine.testutil.testcase.RefreshingTableTestCase.simulateShiftAwareStep;
+
+@Category(OutOfBandTest.class)
+public class TestRollingCountWhere extends BaseUpdateByTest {
+ final int STATIC_TABLE_SIZE = 10_000;
+ final int DYNAMIC_TABLE_SIZE = 1_000;
+ final int DYNAMIC_UPDATE_SIZE = 100;
+ final int DYNAMIC_UPDATE_STEPS = 20;
+
+ // region Object Helper functions
+ private static class TestHelper {
+
+ private static long countWhereInt(final IntVector intVector, final Predicate.Int predicate) {
+ if (intVector == null || intVector.isEmpty()) {
+ return 0L;
+ }
+
+ final long n = intVector.size();
+ long count = 0;
+
+ for (long i = 0; i < n; i++) {
+ if (predicate.call(intVector.get(i))) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ private static long countWhereObject(final ObjectVector objectVector,
+ final Predicate.Unary predicate) {
+ if (objectVector == null || objectVector.isEmpty()) {
+ return 0L;
+ }
+
+ final long n = objectVector.size();
+ long count = 0;
+
+ for (long i = 0; i < n; i++) {
+ if (predicate.call(objectVector.get(i))) {
+ count++;
+ }
+ }
+ return count;
+ }
+ }
+ // endregion Object Helper functions
+
+ // region Static Zero Key Tests
+ @Test
+ public void testStaticZeroKeyAllNullVector() {
+ final int prevTicks = 1;
+ final int postTicks = 0;
+
+ doTestStaticZeroKey(prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticZeroKeyRev() {
+ final int prevTicks = 10;
+ final int postTicks = 0;
+
+ doTestStaticZeroKey(prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticZeroKeyRevExclusive() {
+ final int prevTicks = 10;
+ final int postTicks = -5;
+
+ doTestStaticZeroKey(prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticZeroKeyFwd() {
+ final int prevTicks = 0;
+ final int postTicks = 100;
+
+ doTestStaticZeroKey(prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticZeroKeyFwdExclusive() {
+ final int prevTicks = -50;
+ final int postTicks = 100;
+
+ doTestStaticZeroKey(prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticZeroKeyFwdRevWindow() {
+ final int prevTicks = 100;
+ final int postTicks = 100;
+
+ doTestStaticZeroKey(prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticZeroKeyTimedRev() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ZERO;
+
+ doTestStaticZeroKeyTimed(prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticZeroKeyTimedRevExclusive() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(-5);
+
+ doTestStaticZeroKeyTimed(prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticZeroKeyTimedFwd() {
+ final Duration prevTime = Duration.ZERO;
+ final Duration postTime = Duration.ofMinutes(15);
+
+ doTestStaticZeroKeyTimed(prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticZeroKeyTimedFwdExclusive() {
+ final Duration prevTime = Duration.ofMinutes(-5);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestStaticZeroKeyTimed(prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticZeroKeyTimedFwdRev() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestStaticZeroKeyTimed(prevTime, postTime);
+ }
+
+ private void doTestStaticZeroKey(final int prevTicks, final int postTicks) {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, false, false, 0x31313131,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)}).t;
+
+ Table actual;
+ Table expected;
+
+ // Test simple ChunkFilter, int > 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val > 50));
+ }
+ }
+
+ // Test simple ChunkFilter, int <= 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol <= 50"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val <= 50));
+ }
+ }
+
+ // Test simple conditional filter, true
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "true"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test simple conditional filter, false
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "true"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10 && intCol <= 50"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val > 10 && val <= 50));
+ }
+ }
+
+ // Test chunk, then conditional filter, int > 10 && int % 2 == 0
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10", "intCol % 2 == 0"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val > 10 && val % 2 == 0));
+ }
+ }
+
+ // Test on String column (representing all Object)
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "Sym != null && Sym.startsWith(`A`)"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "SymGroup=Sym"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator> expectedGroupIt = expected.columnIterator("SymGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final ObjectVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereObject(expectedValGroup,
+ val -> val != null && val.startsWith("A")));
+ }
+ }
+
+ // Test OR filter (processed as a WhereFilter)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ Filter.or(Filter.from("intCol < 25", "intCol > 75"))));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val < 25 || val > 75));
+ }
+ }
+
+ // Test AND of chunkfilter and OR filter
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ Filter.and(Filter.and(Filter.from("intCol > 50")),
+ Filter.or(Filter.from("intCol < 25", "intCol > 75")))));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val > 50 && (val < 25 || val > 75)));
+ }
+ }
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))));
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol", "longColGroup=longCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if ((intVal < 25 || intVal > 75) && (longVal < 25 || longVal > 75)) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10", "longCol <= 50"));
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol", "longColGroup=longCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 && longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10 || longCol <= 50"));
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol", "longColGroup=longCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 || longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", filter));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"));
+
+ // The set table contains 0-29, so we can compare against an equivalent filter.
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val >= 0 && val < 30));
+ }
+ }
+
+ // Test Boolean column
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "!isNull(boolCol) && boolCol"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "boolColGroup=boolCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator> expectedGroupIt =
+ expected.columnIterator("boolColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final ObjectVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereObject(expectedValGroup,
+ val -> val != null && val));
+ }
+ }
+
+ // Test Instant column
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "tsGroup=ts"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator> expectedGroupIt = expected.columnIterator("tsGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final ObjectVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereObject(expectedValGroup,
+ val -> val.isAfter(DateTimeUtils.parseInstant("2022-03-09T12:00:00.000 NY"))));
+ }
+ }
+ }
+
+ private void doTestStaticZeroKeyTimed(final Duration prevTime, final Duration postTime) {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, false, false, 0xFFFABBBC,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)}).t;
+
+ Table actual;
+ Table expected;
+
+ // Test simple ChunkFilter, int > 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val > 50));
+ }
+ }
+
+ // Test simple ChunkFilter, int <= 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol <= 50"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val <= 50));
+ }
+ }
+
+ // Test simple conditional filter, true
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "true"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test simple conditional filter, false
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "false"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> false));
+ }
+ }
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 10 && intCol <= 50"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val > 10 && val <= 50));
+ }
+ }
+
+ // Test on String column (representing all Object)
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "Sym != null && Sym.startsWith(`A`)"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "SymGroup=Sym"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator> expectedGroupIt = expected.columnIterator("SymGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final ObjectVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereObject(expectedValGroup,
+ val -> val != null && val.startsWith("A")));
+ }
+ }
+
+ // Test OR filter (processed as a WhereFilter)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ Filter.or(Filter.from("intCol < 25", "intCol > 75"))));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val < 25 || val > 75));
+ }
+ }
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))));
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol", "longColGroup=longCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if ((intVal < 25 || intVal > 75) && (longVal < 25 || longVal > 75)) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 10", "longCol <= 50"));
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol", "longColGroup=longCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 && longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 10 || longCol <= 50"));
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol", "longColGroup=longCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 || longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", filter));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"));
+
+ // The set table contains 0-29, so we can compare against an equivalent filter.
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val >= 0 && val < 30));
+ }
+ }
+
+ // Test Boolean column
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "!isNull(boolCol) && boolCol"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "boolColGroup=boolCol"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator> expectedGroupIt =
+ expected.columnIterator("boolColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final ObjectVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereObject(expectedValGroup,
+ val -> val != null && val));
+ }
+ }
+
+ // Test Instant column
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "ts > DateTimeUtils.parseInstant(`2022-03-09T12:00:00.000 NY`)"));
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "tsGroup=ts"));
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator> expectedGroupIt = expected.columnIterator("tsGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final ObjectVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereObject(expectedValGroup,
+ val -> val.isAfter(DateTimeUtils.parseInstant("2022-03-09T12:00:00.000 NY"))));
+ }
+ }
+ }
+
+ // endregion
+
+ // region Static Bucketed Tests
+
+ @Test
+ public void testStaticGroupedBucketed() {
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ doTestStaticBucketed(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticGroupedBucketedTimed() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(0);
+
+ doTestStaticBucketedTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticBucketedRev() {
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ doTestStaticBucketed(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticBucketedRevExclusive() {
+ final int prevTicks = 100;
+ final int postTicks = -50;
+
+ doTestStaticBucketed(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticBucketedFwd() {
+ final int prevTicks = 0;
+ final int postTicks = 100;
+
+ doTestStaticBucketed(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticBucketedFwdExclusive() {
+ final int prevTicks = -50;
+ final int postTicks = 100;
+
+ doTestStaticBucketed(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testStaticBucketedTimedRev() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(0);
+
+ doTestStaticBucketedTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticBucketedTimedRevExclusive() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(-5);
+
+ doTestStaticBucketedTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticBucketedTimedFwd() {
+ final Duration prevTime = Duration.ofMinutes(0);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestStaticBucketedTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticBucketedTimedFwdExclusive() {
+ final Duration prevTime = Duration.ofMinutes(-5);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestStaticBucketedTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testStaticBucketedFwdRevWindowTimed() {
+ final Duration prevTime = Duration.ofMinutes(5);
+ final Duration postTime = Duration.ofMinutes(5);
+
+ doTestStaticBucketedTimed(false, prevTime, postTime);
+ }
+
+ private void doTestStaticBucketed(boolean grouped, int prevTicks, int postTicks) {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, grouped, false, 0x31313131,
+ new String[] {"charCol"},
+ new TestDataGenerator[] {new CharGenerator('A', 'z', 0.1)}).t;
+
+ Table actual;
+ Table expected;
+
+ // Test simple ChunkFilter, int > 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val > 50));
+ }
+ }
+
+ // Test simple ChunkFilter, int <= 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol <= 50"), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val <= 50));
+ }
+ }
+
+ // Test simple conditional filter, true
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "true"), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test simple conditional filter, false
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "true"), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10 && intCol <= 50"),
+ "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val > 10 && val <= 50));
+ }
+ }
+
+ // Test OR filter (processed as a WhereFilter)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ Filter.or(Filter.from("intCol < 25", "intCol > 75"))), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val < 25 || val > 75));
+ }
+ }
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))),
+ "Sym");
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol", "longColGroup=longCol"),
+ "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if ((intVal < 25 || intVal > 75) && (longVal < 25 || longVal > 75)) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10", "longCol <= 50"),
+ "Sym");
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol", "longColGroup=longCol"),
+ "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 && longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 10 || longCol <= 50"),
+ "Sym");
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol", "longColGroup=longCol"),
+ "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 || longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", filter), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup(prevTicks, postTicks, "intColGroup=intCol"), "Sym");
+
+ // The set table contains 0-29, so we can compare against an equivalent filter.
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val >= 0 && val < 30));
+ }
+ }
+ }
+
+ private void doTestStaticBucketedTimed(boolean grouped, Duration prevTime, Duration postTime) {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, grouped, false, 0xFFFABBBC,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)}).t;
+
+ Table actual;
+ Table expected;
+
+ // Test simple ChunkFilter, int > 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"),
+ "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val > 50));
+ }
+ }
+
+ // Test simple ChunkFilter, int <= 50
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol <= 50"),
+ "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> val <= 50));
+ }
+ }
+
+ // Test simple conditional filter, true
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "true"), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test simple conditional filter, false
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "true"), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", TestHelper.countWhereInt(expectedValGroup, val -> true));
+ }
+ }
+
+ // Test complex conditional filter, int > 10 && int <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 10 && intCol <= 50"),
+ "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val > 10 && val <= 50));
+ }
+ }
+
+ // Test OR filter (processed as a WhereFilter)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ Filter.or(Filter.from("intCol < 25", "intCol > 75"))), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val < 25 || val > 75));
+ }
+ }
+
+ // Test ANDing two OR filter (processed as sequential WhereFilters)
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ Filter.and(Filter.or(Filter.from("intCol < 25", "intCol > 75")),
+ Filter.or(Filter.from("longCol < 25", "longCol > 75")))),
+ "Sym");
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol", "longColGroup=longCol"),
+ "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if ((intVal < 25 || intVal > 75) && (longVal < 25 || longVal > 75)) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column chunk filter, int > 10, longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 10", "longCol <= 50"),
+ "Sym");
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol", "longColGroup=longCol"),
+ "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 && longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test multi-column conditional filter, int > 10 || longCol <= 50
+ actual = t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 10 || longCol <= 50"),
+ "Sym");
+ expected = t.updateBy(
+ UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol", "longColGroup=longCol"),
+ "Sym");
+
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroup1It = expected.columnIterator("intColGroup");
+ final CloseableIterator expectedGroup2It = expected.columnIterator("longColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup1 = expectedGroup1It.next();
+ final LongVector expectedValGroup2 = expectedGroup2It.next();
+
+ long expectedVal = 0;
+ for (int ii = 0; ii < expectedValGroup1.size(); ii++) {
+ final int intVal = expectedValGroup1.get(ii);
+ final long longVal = expectedValGroup2.get(ii);
+
+ if (intVal > 10 || longVal <= 50) {
+ expectedVal++;
+ }
+ }
+
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match", expectedVal);
+ }
+ }
+
+ // Test DynamicWhereFilter to ensure we have supported generic WhereFilters correctly.
+ final QueryTable setTable = (QueryTable) TableTools.emptyTable(30).update("intCol = (int)ii");
+ final DynamicWhereFilter filter = new DynamicWhereFilter(setTable, true, new MatchPair("intCol", "intCol"));
+ actual = t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", filter), "Sym");
+ expected = t.updateBy(UpdateByOperation.RollingGroup("ts", prevTime, postTime, "intColGroup=intCol"), "Sym");
+
+ // The set table contains 0-29, so we can compare against an equivalent filter.
+ try (final CloseablePrimitiveIteratorOfLong actualIt = actual.longColumnIterator("count");
+ final CloseableIterator expectedGroupIt = expected.columnIterator("intColGroup")) {
+ while (actualIt.hasNext()) {
+ final long actualVal = actualIt.nextLong();
+ final IntVector expectedValGroup = expectedGroupIt.next();
+ // Use a lambda over expectedValGroup to compute the expected val.
+ Assert.eq(actualVal, "values match",
+ TestHelper.countWhereInt(expectedValGroup, val -> val >= 0 && val < 30));
+ }
+ }
+ }
+
+ // endregion
+
+ // region Append Only Tests
+
+ @Test
+ public void testZeroKeyAppendOnlyRev() {
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ doTestAppendOnly(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyRevExclusive() {
+ final int prevTicks = 100;
+ final int postTicks = -50;
+
+ doTestAppendOnly(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyFwd() {
+ final int prevTicks = 0;
+ final int postTicks = 100;
+
+ doTestAppendOnly(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyFwdExclusive() {
+ final int prevTicks = -50;
+ final int postTicks = 100;
+
+ doTestAppendOnly(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyFwdRev() {
+ final int prevTicks = 50;
+ final int postTicks = 50;
+
+ doTestAppendOnly(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyRev() {
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ doTestAppendOnly(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyRevExclusive() {
+ final int prevTicks = 100;
+ final int postTicks = -50;
+
+ doTestAppendOnly(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyFwd() {
+ final int prevTicks = 0;
+ final int postTicks = 100;
+
+ doTestAppendOnly(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyFwdExclusive() {
+ final int prevTicks = -50;
+ final int postTicks = 100;
+
+ doTestAppendOnly(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyTimedRev() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(0);
+
+ doTestAppendOnlyTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyTimedRevExclusive() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(-5);
+
+ doTestAppendOnlyTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyTimedFwd() {
+ final Duration prevTime = Duration.ofMinutes(0);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestAppendOnlyTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyTimedFwdExclusive() {
+ final Duration prevTime = Duration.ofMinutes(-5);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestAppendOnlyTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testZeroKeyAppendOnlyTimedFwdRev() {
+ final Duration prevTime = Duration.ofMinutes(5);
+ final Duration postTime = Duration.ofMinutes(5);
+
+ doTestAppendOnlyTimed(false, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyTimedRev() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(0);
+
+ doTestAppendOnlyTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyTimedRevExclusive() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(-5);
+
+ doTestAppendOnlyTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyTimedFwd() {
+ final Duration prevTime = Duration.ofMinutes(0);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestAppendOnlyTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyTimedFwdExclusive() {
+ final Duration prevTime = Duration.ofMinutes(-5);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestAppendOnlyTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedAppendOnlyTimedFwdRev() {
+ final Duration prevTime = Duration.ofMinutes(5);
+ final Duration postTime = Duration.ofMinutes(5);
+
+ doTestAppendOnlyTimed(true, prevTime, postTime);
+ }
+
+ private void doTestAppendOnly(boolean bucketed, int prevTicks, int postTicks) {
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, bucketed, false, true, 0x31313131,
+ new String[] {"charCol"},
+ new TestDataGenerator[] {new CharGenerator('A', 'z', 0.1)});
+ final QueryTable t = result.t;
+ t.setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"),
+ "Sym")
+ : t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "false"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "false"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "longCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && longCol < 90")))
+ };
+
+ final Random billy = new Random(0xB177B177);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle(
+ () -> generateAppends(DYNAMIC_UPDATE_SIZE, billy, t, result.infos));
+ TstUtils.validate("Table", nuggets);
+ }
+ }
+
+ private void doTestAppendOnlyTimed(boolean bucketed, Duration prevTime, Duration postTime) {
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, bucketed, false, true, 0x31313131,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)});
+ final QueryTable t = result.t;
+ t.setAttribute(Table.APPEND_ONLY_TABLE_ATTRIBUTE, Boolean.TRUE);
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"),
+ "Sym")
+ : t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "false"),
+ "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "false"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "longCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && longCol < 90")))
+ };
+
+ final Random billy = new Random(0xB177B177);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle(
+ () -> generateAppends(DYNAMIC_UPDATE_SIZE, billy, t, result.infos));
+ TstUtils.validate("Table", nuggets);
+ }
+ }
+
+ // endregion Append Only Tests
+
+ // region General Ticking Tests
+
+ @Test
+ public void testZeroKeyGeneralTickingRev() {
+ final long prevTicks = 100;
+ final long fwdTicks = 0;
+
+ doTestTicking(false, prevTicks, fwdTicks);
+ }
+
+ @Test
+ public void testZeroKeyGeneralTickingRevExclusive() {
+ final long prevTicks = 100;
+ final long fwdTicks = -50;
+
+ doTestTicking(false, prevTicks, fwdTicks);
+ }
+
+ @Test
+ public void testZeroKeyGeneralTickingFwd() {
+ final long prevTicks = 0;
+ final long fwdTicks = 100;
+
+ doTestTicking(false, prevTicks, fwdTicks);
+ }
+
+ @Test
+ public void testZeroKeyGeneralTickingFwdExclusive() {
+ final long prevTicks = -50;
+ final long fwdTicks = 100;
+
+ doTestTicking(false, prevTicks, fwdTicks);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingRev() {
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ doTestTicking(false, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingRevExclusive() {
+ final int prevTicks = 100;
+ final int postTicks = -50;
+
+ doTestTicking(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingFwd() {
+ final int prevTicks = 0;
+ final int postTicks = 100;
+
+ doTestTicking(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingFwdExclusive() {
+ final int prevTicks = -50;
+ final int postTicks = 100;
+
+ doTestTicking(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingFwdRev() {
+ final int prevTicks = 50;
+ final int postTicks = 50;
+
+ doTestTicking(true, prevTicks, postTicks);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingTimedRev() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(0);
+
+ doTestTickingTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingTimedRevExclusive() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(-5);
+
+ doTestTickingTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingTimedFwd() {
+ final Duration prevTime = Duration.ofMinutes(0);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestTickingTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingTimedFwdExclusive() {
+ final Duration prevTime = Duration.ofMinutes(-5);
+ final Duration postTime = Duration.ofMinutes(10);
+
+ doTestTickingTimed(true, prevTime, postTime);
+ }
+
+ @Test
+ public void testBucketedGeneralTickingTimedFwdRev() {
+ final Duration prevTime = Duration.ofMinutes(5);
+ final Duration postTime = Duration.ofMinutes(5);
+
+ doTestTickingTimed(true, prevTime, postTime);
+ }
+
+ private void doTestTicking(final boolean bucketed, final long prevTicks, final long postTicks) {
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, bucketed, false, true, 0x31313131,
+ new String[] {"charCol"},
+ new TestDataGenerator[] {new CharGenerator('A', 'z', 0.1)});
+ final QueryTable t = result.t;
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"),
+ "Sym")
+ : t.updateBy(
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "false"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "false"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "longCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && longCol < 90")))
+ };
+
+ final Random billy = new Random(0xB177B177);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle(
+ () -> GenerateTableUpdates.generateTableUpdates(DYNAMIC_UPDATE_SIZE, billy, t, result.infos));
+ TstUtils.validate("Table - step " + ii, nuggets);
+ }
+ }
+
+ private void doTestTickingTimed(final boolean bucketed, final Duration prevTime, final Duration postTime) {
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, bucketed, false, true, 0x31313131,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)});
+
+ final QueryTable t = result.t;
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"),
+ "Sym")
+ : t.updateBy(
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && intCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && intCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "false"),
+ "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "false"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50", "longCol < 90"))),
+ EvalNugget.from(() -> bucketed
+ ? t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && longCol < 90"), "Sym")
+ : t.updateBy(UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && longCol < 90")))
+ };
+
+
+ final Random billy = new Random(0xB177B177);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ ExecutionContext.getContext().getUpdateGraph().cast().runWithinUnitTestCycle(
+ () -> GenerateTableUpdates.generateTableUpdates(DYNAMIC_UPDATE_SIZE, billy, t, result.infos));
+ TstUtils.validate("Table - step " + ii, nuggets);
+ }
+ }
+
+ @Test
+ public void testBucketedGeneralTickingRevRedirected() {
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, true, false, true, 0x31313131,
+ new String[] {"charCol"},
+ new TestDataGenerator[] {new CharGenerator('A', 'z', 0.1)});
+ final QueryTable t = result.t;
+
+ final UpdateByControl control = UpdateByControl.builder().useRedirection(true).build();
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"), "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "intCol < 90"),
+ "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && intCol < 90"),
+ "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "false"), "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50",
+ "longCol < 90"),
+ "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count",
+ "intCol > 50 && longCol < 90"),
+ "Sym"))
+ };
+
+ final Random billy = new Random(0xB177B177);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ try {
+ simulateShiftAwareStep(DYNAMIC_UPDATE_SIZE, billy, t, result.infos, nuggets);
+ } catch (Throwable ex) {
+ System.out.println("Crapped out on step " + ii);
+ throw ex;
+ }
+ }
+ }
+
+ @Test
+ public void testBucketedGeneralTickingTimedRevRedirected() {
+ final Duration prevTime = Duration.ofMinutes(10);
+ final Duration postTime = Duration.ofMinutes(0);
+
+ final CreateResult result = createTestTable(DYNAMIC_TABLE_SIZE, true, false, true, 0x31313131,
+ new String[] {"ts", "charCol"}, new TestDataGenerator[] {new SortedInstantGenerator(
+ DateTimeUtils.parseInstant("2022-03-09T09:00:00.000 NY"),
+ DateTimeUtils.parseInstant("2022-03-09T16:30:00.000 NY")),
+ new CharGenerator('A', 'z', 0.1)});
+
+ final QueryTable t = result.t;
+
+ final UpdateByControl control = UpdateByControl.builder().useRedirection(true).build();
+
+ final EvalNugget[] nuggets = new EvalNugget[] {
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50"), "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50",
+ "intCol < 90"),
+ "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && intCol < 90"),
+ "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "false"), "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count", "intCol > 50",
+ "longCol < 90"),
+ "Sym")),
+ EvalNugget.from(() -> t.updateBy(control,
+ UpdateByOperation.RollingCountWhere("ts", prevTime, postTime, "count",
+ "intCol > 50 && longCol < 90"),
+ "Sym"))
+ };
+
+
+ final Random billy = new Random(0xB177B177);
+ for (int ii = 0; ii < DYNAMIC_UPDATE_STEPS; ii++) {
+ try {
+ simulateShiftAwareStep(DYNAMIC_UPDATE_SIZE, billy, t, result.infos, nuggets);
+ } catch (Throwable ex) {
+ System.out.println("Crapped out on step " + ii);
+ throw ex;
+ }
+ }
+ }
+
+ // endregion
+
+ @Test
+ public void testProxy() {
+ final QueryTable t = createTestTable(STATIC_TABLE_SIZE, true, false, false, 0x31313131).t;
+
+ final int prevTicks = 100;
+ final int postTicks = 0;
+
+ Table actual;
+ Table expected;
+
+ // Compare the merged proxy table to the bucketed version (need to sort by symbol to get alignment).
+ PartitionedTable pt = t.partitionBy("Sym");
+ actual = pt.proxy()
+ .updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"))
+ .target().merge().sort("Sym");
+ expected = t
+ .updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol > 50"), "Sym")
+ .sort("Sym");
+ assertTableEquals(expected, actual);
+
+ actual = pt.proxy()
+ .updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol <= 50"))
+ .target().merge().sort("Sym");
+ expected = t
+ .updateBy(UpdateByOperation.RollingCountWhere(prevTicks, postTicks, "count", "intCol <= 50"), "Sym")
+ .sort("Sym");
+ assertTableEquals(expected, actual);
+ }
+}
diff --git a/go/internal/proto/table/table.pb.go b/go/internal/proto/table/table.pb.go
index fe6f76ab686..1130464a7a7 100644
--- a/go/internal/proto/table/table.pb.go
+++ b/go/internal/proto/table/table.pb.go
@@ -6236,6 +6236,8 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec struct {
// *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingStd
// *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingWavg
// *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingFormula
+ // *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingCountWhere
+ // *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_CountWhere
Type isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type `protobuf_oneof:"type"`
}
@@ -6425,6 +6427,20 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) GetRolli
return nil
}
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) GetRollingCountWhere() *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere {
+ if x, ok := x.GetType().(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingCountWhere); ok {
+ return x.RollingCountWhere
+ }
+ return nil
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec) GetCountWhere() *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere {
+ if x, ok := x.GetType().(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_CountWhere); ok {
+ return x.CountWhere
+ }
+ return nil
+}
+
type isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type interface {
isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type()
}
@@ -6513,6 +6529,14 @@ type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingFormul
RollingFormula *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingFormula `protobuf:"bytes,21,opt,name=rolling_formula,json=rollingFormula,proto3,oneof"`
}
+type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingCountWhere struct {
+ RollingCountWhere *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere `protobuf:"bytes,22,opt,name=rolling_count_where,json=rollingCountWhere,proto3,oneof"`
+}
+
+type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_CountWhere struct {
+ CountWhere *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere `protobuf:"bytes,23,opt,name=count_where,json=countWhere,proto3,oneof"`
+}
+
func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Sum) isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type() {
}
@@ -6576,6 +6600,12 @@ func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingWavg
func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingFormula) isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type() {
}
+func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingCountWhere) isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type() {
+}
+
+func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_CountWhere) isUpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_Type() {
+}
+
type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeSum struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -7677,6 +7707,140 @@ func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByR
return ""
}
+type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ // The reverse window for the rolling operation, may be in ticks or time-based.
+ ReverseWindowScale *UpdateByWindowScale `protobuf:"bytes,1,opt,name=reverse_window_scale,json=reverseWindowScale,proto3" json:"reverse_window_scale,omitempty"`
+ // The forward window for the rolling operation, may be in ticks or time-based.
+ ForwardWindowScale *UpdateByWindowScale `protobuf:"bytes,2,opt,name=forward_window_scale,json=forwardWindowScale,proto3" json:"forward_window_scale,omitempty"`
+ // The output column name
+ ResultColumn string `protobuf:"bytes,3,opt,name=result_column,json=resultColumn,proto3" json:"result_column,omitempty"`
+ // The filters tp apply before counting
+ Filters []string `protobuf:"bytes,4,rep,name=filters,proto3" json:"filters,omitempty"`
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) Reset() {
+ *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[92]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) ProtoMessage() {
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) ProtoReflect() protoreflect.Message {
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[92]
+ 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 UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere.ProtoReflect.Descriptor instead.
+func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) Descriptor() ([]byte, []int) {
+ return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 21}
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) GetReverseWindowScale() *UpdateByWindowScale {
+ if x != nil {
+ return x.ReverseWindowScale
+ }
+ return nil
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) GetForwardWindowScale() *UpdateByWindowScale {
+ if x != nil {
+ return x.ForwardWindowScale
+ }
+ return nil
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) GetResultColumn() string {
+ if x != nil {
+ return x.ResultColumn
+ }
+ return ""
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere) GetFilters() []string {
+ if x != nil {
+ return x.Filters
+ }
+ return nil
+}
+
+type UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere struct {
+ state protoimpl.MessageState
+ sizeCache protoimpl.SizeCache
+ unknownFields protoimpl.UnknownFields
+
+ // The output column name
+ ResultColumn string `protobuf:"bytes,1,opt,name=result_column,json=resultColumn,proto3" json:"result_column,omitempty"`
+ // The filters tp apply before counting
+ Filters []string `protobuf:"bytes,2,rep,name=filters,proto3" json:"filters,omitempty"`
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) Reset() {
+ *x = UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere{}
+ if protoimpl.UnsafeEnabled {
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[93]
+ ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+ ms.StoreMessageInfo(mi)
+ }
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) String() string {
+ return protoimpl.X.MessageStringOf(x)
+}
+
+func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) ProtoMessage() {
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) ProtoReflect() protoreflect.Message {
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[93]
+ 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 UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere.ProtoReflect.Descriptor instead.
+func (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) Descriptor() ([]byte, []int) {
+ return file_deephaven_core_proto_table_proto_rawDescGZIP(), []int{14, 1, 0, 0, 22}
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) GetResultColumn() string {
+ if x != nil {
+ return x.ResultColumn
+ }
+ return ""
+}
+
+func (x *UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere) GetFilters() []string {
+ if x != nil {
+ return x.Filters
+ }
+ return nil
+}
+
type ComboAggregateRequest_Aggregate struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -7692,7 +7856,7 @@ type ComboAggregateRequest_Aggregate struct {
func (x *ComboAggregateRequest_Aggregate) Reset() {
*x = ComboAggregateRequest_Aggregate{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[92]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[94]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -7705,7 +7869,7 @@ func (x *ComboAggregateRequest_Aggregate) String() string {
func (*ComboAggregateRequest_Aggregate) ProtoMessage() {}
func (x *ComboAggregateRequest_Aggregate) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[92]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[94]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -7771,7 +7935,7 @@ type AggSpec_AggSpecApproximatePercentile struct {
func (x *AggSpec_AggSpecApproximatePercentile) Reset() {
*x = AggSpec_AggSpecApproximatePercentile{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[93]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[95]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -7784,7 +7948,7 @@ func (x *AggSpec_AggSpecApproximatePercentile) String() string {
func (*AggSpec_AggSpecApproximatePercentile) ProtoMessage() {}
func (x *AggSpec_AggSpecApproximatePercentile) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[93]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[95]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -7826,7 +7990,7 @@ type AggSpec_AggSpecCountDistinct struct {
func (x *AggSpec_AggSpecCountDistinct) Reset() {
*x = AggSpec_AggSpecCountDistinct{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[94]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[96]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -7839,7 +8003,7 @@ func (x *AggSpec_AggSpecCountDistinct) String() string {
func (*AggSpec_AggSpecCountDistinct) ProtoMessage() {}
func (x *AggSpec_AggSpecCountDistinct) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[94]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[96]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -7874,7 +8038,7 @@ type AggSpec_AggSpecDistinct struct {
func (x *AggSpec_AggSpecDistinct) Reset() {
*x = AggSpec_AggSpecDistinct{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[95]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[97]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -7887,7 +8051,7 @@ func (x *AggSpec_AggSpecDistinct) String() string {
func (*AggSpec_AggSpecDistinct) ProtoMessage() {}
func (x *AggSpec_AggSpecDistinct) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[95]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[97]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -7924,7 +8088,7 @@ type AggSpec_AggSpecFormula struct {
func (x *AggSpec_AggSpecFormula) Reset() {
*x = AggSpec_AggSpecFormula{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[96]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[98]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -7937,7 +8101,7 @@ func (x *AggSpec_AggSpecFormula) String() string {
func (*AggSpec_AggSpecFormula) ProtoMessage() {}
func (x *AggSpec_AggSpecFormula) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[96]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[98]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -7980,7 +8144,7 @@ type AggSpec_AggSpecMedian struct {
func (x *AggSpec_AggSpecMedian) Reset() {
*x = AggSpec_AggSpecMedian{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[97]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[99]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -7993,7 +8157,7 @@ func (x *AggSpec_AggSpecMedian) String() string {
func (*AggSpec_AggSpecMedian) ProtoMessage() {}
func (x *AggSpec_AggSpecMedian) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[97]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[99]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8031,7 +8195,7 @@ type AggSpec_AggSpecPercentile struct {
func (x *AggSpec_AggSpecPercentile) Reset() {
*x = AggSpec_AggSpecPercentile{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[98]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[100]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8044,7 +8208,7 @@ func (x *AggSpec_AggSpecPercentile) String() string {
func (*AggSpec_AggSpecPercentile) ProtoMessage() {}
func (x *AggSpec_AggSpecPercentile) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[98]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[100]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8086,7 +8250,7 @@ type AggSpec_AggSpecSorted struct {
func (x *AggSpec_AggSpecSorted) Reset() {
*x = AggSpec_AggSpecSorted{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[99]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[101]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8099,7 +8263,7 @@ func (x *AggSpec_AggSpecSorted) String() string {
func (*AggSpec_AggSpecSorted) ProtoMessage() {}
func (x *AggSpec_AggSpecSorted) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[99]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[101]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8134,7 +8298,7 @@ type AggSpec_AggSpecSortedColumn struct {
func (x *AggSpec_AggSpecSortedColumn) Reset() {
*x = AggSpec_AggSpecSortedColumn{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[100]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[102]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8147,7 +8311,7 @@ func (x *AggSpec_AggSpecSortedColumn) String() string {
func (*AggSpec_AggSpecSortedColumn) ProtoMessage() {}
func (x *AggSpec_AggSpecSortedColumn) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[100]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[102]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8183,7 +8347,7 @@ type AggSpec_AggSpecTDigest struct {
func (x *AggSpec_AggSpecTDigest) Reset() {
*x = AggSpec_AggSpecTDigest{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[101]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[103]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8196,7 +8360,7 @@ func (x *AggSpec_AggSpecTDigest) String() string {
func (*AggSpec_AggSpecTDigest) ProtoMessage() {}
func (x *AggSpec_AggSpecTDigest) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[101]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[103]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8233,7 +8397,7 @@ type AggSpec_AggSpecUnique struct {
func (x *AggSpec_AggSpecUnique) Reset() {
*x = AggSpec_AggSpecUnique{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[102]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[104]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8246,7 +8410,7 @@ func (x *AggSpec_AggSpecUnique) String() string {
func (*AggSpec_AggSpecUnique) ProtoMessage() {}
func (x *AggSpec_AggSpecUnique) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[102]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[104]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8298,7 +8462,7 @@ type AggSpec_AggSpecNonUniqueSentinel struct {
func (x *AggSpec_AggSpecNonUniqueSentinel) Reset() {
*x = AggSpec_AggSpecNonUniqueSentinel{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[103]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[105]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8311,7 +8475,7 @@ func (x *AggSpec_AggSpecNonUniqueSentinel) String() string {
func (*AggSpec_AggSpecNonUniqueSentinel) ProtoMessage() {}
func (x *AggSpec_AggSpecNonUniqueSentinel) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[103]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[105]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8483,7 +8647,7 @@ type AggSpec_AggSpecWeighted struct {
func (x *AggSpec_AggSpecWeighted) Reset() {
*x = AggSpec_AggSpecWeighted{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[104]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[106]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8496,7 +8660,7 @@ func (x *AggSpec_AggSpecWeighted) String() string {
func (*AggSpec_AggSpecWeighted) ProtoMessage() {}
func (x *AggSpec_AggSpecWeighted) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[104]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[106]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8528,7 +8692,7 @@ type AggSpec_AggSpecAbsSum struct {
func (x *AggSpec_AggSpecAbsSum) Reset() {
*x = AggSpec_AggSpecAbsSum{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[105]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[107]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8541,7 +8705,7 @@ func (x *AggSpec_AggSpecAbsSum) String() string {
func (*AggSpec_AggSpecAbsSum) ProtoMessage() {}
func (x *AggSpec_AggSpecAbsSum) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[105]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[107]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8566,7 +8730,7 @@ type AggSpec_AggSpecAvg struct {
func (x *AggSpec_AggSpecAvg) Reset() {
*x = AggSpec_AggSpecAvg{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[106]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[108]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8579,7 +8743,7 @@ func (x *AggSpec_AggSpecAvg) String() string {
func (*AggSpec_AggSpecAvg) ProtoMessage() {}
func (x *AggSpec_AggSpecAvg) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[106]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[108]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8604,7 +8768,7 @@ type AggSpec_AggSpecFirst struct {
func (x *AggSpec_AggSpecFirst) Reset() {
*x = AggSpec_AggSpecFirst{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[107]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[109]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8617,7 +8781,7 @@ func (x *AggSpec_AggSpecFirst) String() string {
func (*AggSpec_AggSpecFirst) ProtoMessage() {}
func (x *AggSpec_AggSpecFirst) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[107]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[109]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8642,7 +8806,7 @@ type AggSpec_AggSpecFreeze struct {
func (x *AggSpec_AggSpecFreeze) Reset() {
*x = AggSpec_AggSpecFreeze{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[108]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[110]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8655,7 +8819,7 @@ func (x *AggSpec_AggSpecFreeze) String() string {
func (*AggSpec_AggSpecFreeze) ProtoMessage() {}
func (x *AggSpec_AggSpecFreeze) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[108]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[110]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8680,7 +8844,7 @@ type AggSpec_AggSpecGroup struct {
func (x *AggSpec_AggSpecGroup) Reset() {
*x = AggSpec_AggSpecGroup{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[109]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[111]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8693,7 +8857,7 @@ func (x *AggSpec_AggSpecGroup) String() string {
func (*AggSpec_AggSpecGroup) ProtoMessage() {}
func (x *AggSpec_AggSpecGroup) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[109]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[111]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8718,7 +8882,7 @@ type AggSpec_AggSpecLast struct {
func (x *AggSpec_AggSpecLast) Reset() {
*x = AggSpec_AggSpecLast{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[110]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[112]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8731,7 +8895,7 @@ func (x *AggSpec_AggSpecLast) String() string {
func (*AggSpec_AggSpecLast) ProtoMessage() {}
func (x *AggSpec_AggSpecLast) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[110]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[112]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8756,7 +8920,7 @@ type AggSpec_AggSpecMax struct {
func (x *AggSpec_AggSpecMax) Reset() {
*x = AggSpec_AggSpecMax{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[111]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[113]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8769,7 +8933,7 @@ func (x *AggSpec_AggSpecMax) String() string {
func (*AggSpec_AggSpecMax) ProtoMessage() {}
func (x *AggSpec_AggSpecMax) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[111]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[113]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8794,7 +8958,7 @@ type AggSpec_AggSpecMin struct {
func (x *AggSpec_AggSpecMin) Reset() {
*x = AggSpec_AggSpecMin{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[112]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[114]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8807,7 +8971,7 @@ func (x *AggSpec_AggSpecMin) String() string {
func (*AggSpec_AggSpecMin) ProtoMessage() {}
func (x *AggSpec_AggSpecMin) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[112]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[114]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8832,7 +8996,7 @@ type AggSpec_AggSpecStd struct {
func (x *AggSpec_AggSpecStd) Reset() {
*x = AggSpec_AggSpecStd{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[113]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[115]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8845,7 +9009,7 @@ func (x *AggSpec_AggSpecStd) String() string {
func (*AggSpec_AggSpecStd) ProtoMessage() {}
func (x *AggSpec_AggSpecStd) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[113]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[115]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8870,7 +9034,7 @@ type AggSpec_AggSpecSum struct {
func (x *AggSpec_AggSpecSum) Reset() {
*x = AggSpec_AggSpecSum{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[114]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[116]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8883,7 +9047,7 @@ func (x *AggSpec_AggSpecSum) String() string {
func (*AggSpec_AggSpecSum) ProtoMessage() {}
func (x *AggSpec_AggSpecSum) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[114]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[116]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8908,7 +9072,7 @@ type AggSpec_AggSpecVar struct {
func (x *AggSpec_AggSpecVar) Reset() {
*x = AggSpec_AggSpecVar{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[115]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[117]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8921,7 +9085,7 @@ func (x *AggSpec_AggSpecVar) String() string {
func (*AggSpec_AggSpecVar) ProtoMessage() {}
func (x *AggSpec_AggSpecVar) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[115]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[117]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -8949,7 +9113,7 @@ type Aggregation_AggregationColumns struct {
func (x *Aggregation_AggregationColumns) Reset() {
*x = Aggregation_AggregationColumns{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[116]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[118]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -8962,7 +9126,7 @@ func (x *Aggregation_AggregationColumns) String() string {
func (*Aggregation_AggregationColumns) ProtoMessage() {}
func (x *Aggregation_AggregationColumns) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[116]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[118]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9004,7 +9168,7 @@ type Aggregation_AggregationCount struct {
func (x *Aggregation_AggregationCount) Reset() {
*x = Aggregation_AggregationCount{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[117]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[119]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9017,7 +9181,7 @@ func (x *Aggregation_AggregationCount) String() string {
func (*Aggregation_AggregationCount) ProtoMessage() {}
func (x *Aggregation_AggregationCount) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[117]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[119]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9053,7 +9217,7 @@ type Aggregation_AggregationCountWhere struct {
func (x *Aggregation_AggregationCountWhere) Reset() {
*x = Aggregation_AggregationCountWhere{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[118]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[120]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9066,7 +9230,7 @@ func (x *Aggregation_AggregationCountWhere) String() string {
func (*Aggregation_AggregationCountWhere) ProtoMessage() {}
func (x *Aggregation_AggregationCountWhere) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[118]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[120]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9107,7 +9271,7 @@ type Aggregation_AggregationRowKey struct {
func (x *Aggregation_AggregationRowKey) Reset() {
*x = Aggregation_AggregationRowKey{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[119]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[121]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9120,7 +9284,7 @@ func (x *Aggregation_AggregationRowKey) String() string {
func (*Aggregation_AggregationRowKey) ProtoMessage() {}
func (x *Aggregation_AggregationRowKey) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[119]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[121]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9155,7 +9319,7 @@ type Aggregation_AggregationPartition struct {
func (x *Aggregation_AggregationPartition) Reset() {
*x = Aggregation_AggregationPartition{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[120]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[122]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9168,7 +9332,7 @@ func (x *Aggregation_AggregationPartition) String() string {
func (*Aggregation_AggregationPartition) ProtoMessage() {}
func (x *Aggregation_AggregationPartition) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[120]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[122]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9209,7 +9373,7 @@ type Aggregation_AggregationFormula struct {
func (x *Aggregation_AggregationFormula) Reset() {
*x = Aggregation_AggregationFormula{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[121]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[123]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9222,7 +9386,7 @@ func (x *Aggregation_AggregationFormula) String() string {
func (*Aggregation_AggregationFormula) ProtoMessage() {}
func (x *Aggregation_AggregationFormula) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[121]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[123]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9257,7 +9421,7 @@ type RunChartDownsampleRequest_ZoomRange struct {
func (x *RunChartDownsampleRequest_ZoomRange) Reset() {
*x = RunChartDownsampleRequest_ZoomRange{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[122]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[124]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9270,7 +9434,7 @@ func (x *RunChartDownsampleRequest_ZoomRange) String() string {
func (*RunChartDownsampleRequest_ZoomRange) ProtoMessage() {}
func (x *RunChartDownsampleRequest_ZoomRange) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[122]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[124]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9315,7 +9479,7 @@ type CreateInputTableRequest_InputTableKind struct {
func (x *CreateInputTableRequest_InputTableKind) Reset() {
*x = CreateInputTableRequest_InputTableKind{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[123]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[125]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9328,7 +9492,7 @@ func (x *CreateInputTableRequest_InputTableKind) String() string {
func (*CreateInputTableRequest_InputTableKind) ProtoMessage() {}
func (x *CreateInputTableRequest_InputTableKind) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[123]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[125]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9407,7 +9571,7 @@ type CreateInputTableRequest_InputTableKind_InMemoryAppendOnly struct {
func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) Reset() {
*x = CreateInputTableRequest_InputTableKind_InMemoryAppendOnly{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[124]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[126]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9420,7 +9584,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) String() str
func (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoMessage() {}
func (x *CreateInputTableRequest_InputTableKind_InMemoryAppendOnly) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[124]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[126]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9448,7 +9612,7 @@ type CreateInputTableRequest_InputTableKind_InMemoryKeyBacked struct {
func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) Reset() {
*x = CreateInputTableRequest_InputTableKind_InMemoryKeyBacked{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[125]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[127]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9461,7 +9625,7 @@ func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) String() stri
func (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoMessage() {}
func (x *CreateInputTableRequest_InputTableKind_InMemoryKeyBacked) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[125]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[127]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9493,7 +9657,7 @@ type CreateInputTableRequest_InputTableKind_Blink struct {
func (x *CreateInputTableRequest_InputTableKind_Blink) Reset() {
*x = CreateInputTableRequest_InputTableKind_Blink{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[126]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[128]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9506,7 +9670,7 @@ func (x *CreateInputTableRequest_InputTableKind_Blink) String() string {
func (*CreateInputTableRequest_InputTableKind_Blink) ProtoMessage() {}
func (x *CreateInputTableRequest_InputTableKind_Blink) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[126]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[128]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -9576,7 +9740,7 @@ type BatchTableRequest_Operation struct {
func (x *BatchTableRequest_Operation) Reset() {
*x = BatchTableRequest_Operation{}
if protoimpl.UnsafeEnabled {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[127]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[129]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@@ -9589,7 +9753,7 @@ func (x *BatchTableRequest_Operation) String() string {
func (*BatchTableRequest_Operation) ProtoMessage() {}
func (x *BatchTableRequest_Operation) ProtoReflect() protoreflect.Message {
- mi := &file_deephaven_core_proto_table_proto_msgTypes[127]
+ mi := &file_deephaven_core_proto_table_proto_msgTypes[129]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@@ -10364,7 +10528,7 @@ var file_deephaven_core_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, 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,
+ 0x72, 0x22, 0xcd, 0x44, 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,
@@ -10426,7 +10590,7 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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,
+ 0x5f, 0x66, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x1a, 0xe7, 0x3c, 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,
@@ -10434,7 +10598,7 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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,
+ 0x6d, 0x6e, 0x48, 0x00, 0x52, 0x06, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x1a, 0xda, 0x3b, 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,
@@ -10445,7 +10609,7 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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,
+ 0x68, 0x50, 0x61, 0x69, 0x72, 0x73, 0x1a, 0xb0, 0x3a, 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,
@@ -10639,63 +10803,37 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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, 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, 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, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x6c, 0x69, 0x6e, 0x67, 0x46, 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0xac, 0x01, 0x0a, 0x13,
+ 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x77, 0x68,
+ 0x65, 0x72, 0x65, 0x18, 0x16, 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, 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,
+ 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,
+ 0x57, 0x68, 0x65, 0x72, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x6f, 0x6c, 0x6c, 0x69, 0x6e, 0x67,
+ 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x12, 0xa0, 0x01, 0x0a, 0x0b, 0x63,
+ 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x77, 0x68, 0x65, 0x72, 0x65, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x7d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x48,
+ 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x1a, 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,
@@ -10706,103 +10844,91 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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,
+ 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, 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, 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, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
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,
+ 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, 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, 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,
+ 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, 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, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 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,
@@ -10816,7 +10942,7 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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,
+ 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,
@@ -10830,58 +10956,129 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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, 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, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 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, 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,
+ 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, 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, 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, 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,
+ 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, 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, 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, 0x1a, 0xae, 0x02, 0x0a, 0x19, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x52, 0x6f,
+ 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 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, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x75,
+ 0x6c, 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74,
+ 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65,
+ 0x72, 0x73, 0x1a, 0x5d, 0x0a, 0x1c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x43, 0x75,
+ 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x76, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65,
+ 0x72, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x63, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65, 0x73, 0x75, 0x6c,
+ 0x74, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65,
+ 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72,
+ 0x73, 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,
@@ -10893,50 +11090,8 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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, 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, 0x70, 0x68, 0x61, 0x76, 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,
+ 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,
@@ -10946,83 +11101,115 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 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, 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, 0x70, 0x68, 0x61, 0x76, 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,
- 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,
+ 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, 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, 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,
+ 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, 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,
+ 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, 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,
@@ -11042,1698 +11229,1721 @@ var file_deephaven_core_proto_table_proto_rawDesc = []byte{
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, 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,
+ 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, 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, 0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 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, 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, 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,
+ 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, 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,
+ 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, 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, 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, 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,
+ 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, 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, 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,
+ 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, 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, 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,
+ 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, 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, 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, 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,
+ 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, 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,
- 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,
+ 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, 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, 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,
+ 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, 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, 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,
+ 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, 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, 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,
+ 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, 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,
+ 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, 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, 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,
+ 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, 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,
+ 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, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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,
- 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,
+ 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, 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,
+ 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, 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,
+ 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, 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,
+ 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, 0xd5, 0x09, 0x0a, 0x0b, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67,
+ 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5d, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
+ 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
+ 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b,
+ 0x70, 0x6c, 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, 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, 0xd5, 0x09, 0x0a, 0x0b, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74,
- 0x69, 0x6f, 0x6e, 0x12, 0x5d, 0x0a, 0x07, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
+ 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,
- 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,
+ 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, 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, 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, 0x12,
+ 0x67, 0x0a, 0x0b, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x77, 0x68, 0x65, 0x72, 0x65, 0x18, 0x07,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
+ 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
+ 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61,
+ 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+ 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f,
+ 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x1a, 0x75, 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72,
+ 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x3e,
+ 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69,
0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63,
- 0x2e, 0x41, 0x67, 0x67, 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, 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, 0x12, 0x67, 0x0a,
- 0x0b, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x5f, 0x77, 0x68, 0x65, 0x72, 0x65, 0x18, 0x07, 0x20, 0x01,
- 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65,
- 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e,
- 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69,
- 0x6f, 0x6e, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f,
- 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x63, 0x6f, 0x75, 0x6e,
- 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x1a, 0x75, 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67,
- 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x3e, 0x0a, 0x04,
- 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69, 0x6f, 0x2e,
+ 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, 0x52, 0x0a, 0x15, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74,
+ 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x12, 0x1f, 0x0a,
+ 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18,
+ 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52,
+ 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x1a, 0x34, 0x0a, 0x11, 0x41, 0x67, 0x67, 0x72,
+ 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, 0x1f, 0x0a,
+ 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
+ 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x70,
+ 0x0a, 0x14, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72,
+ 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,
+ 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c,
+ 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x6e, 0x63, 0x6c, 0x75,
+ 0x64, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x6e, 0x63, 0x6c, 0x75,
+ 0x64, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73,
+ 0x1a, 0x63, 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46,
+ 0x6f, 0x72, 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x4d, 0x0a, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74,
+ 0x61, 0x62, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x69, 0x6f, 0x2e,
0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
- 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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, 0x52, 0x0a, 0x15, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f,
- 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x57, 0x68, 0x65, 0x72, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x63,
- 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07,
- 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x66,
- 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x1a, 0x34, 0x0a, 0x11, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67,
- 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x6f, 0x77, 0x4b, 0x65, 0x79, 0x12, 0x1f, 0x0a, 0x0b, 0x63,
- 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
- 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x70, 0x0a, 0x14,
- 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x72, 0x74, 0x69,
- 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e,
- 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d,
- 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x37, 0x0a, 0x18, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65,
- 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x62, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,
- 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x69, 0x6e, 0x63, 0x6c, 0x75, 0x64, 0x65,
- 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x63,
- 0x0a, 0x12, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x6f, 0x72,
- 0x6d, 0x75, 0x6c, 0x61, 0x12, 0x4d, 0x0a, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61, 0x62,
- 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
- 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
- 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x6c,
- 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61,
- 0x62, 0x6c, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x84, 0x02, 0x0a, 0x0e,
- 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x1f,
- 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12,
- 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x61, 0x62, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x65, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x41, 0x62, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x65,
- 0x12, 0x5d, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20,
- 0x01, 0x28, 0x0e, 0x32, 0x3f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
- 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61,
- 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63,
- 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63,
- 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22,
- 0x51, 0x0a, 0x0d, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
- 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x17, 0x0a,
- 0x0a, 0x44, 0x45, 0x53, 0x43, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0xff, 0xff, 0xff, 0xff,
- 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x41, 0x53, 0x43, 0x45, 0x4e, 0x44,
- 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x56, 0x45, 0x52, 0x53, 0x45,
- 0x10, 0x02, 0x22, 0xf3, 0x01, 0x0a, 0x10, 0x53, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c,
- 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53,
+ 0x65, 0x6c, 0x65, 0x63, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x0a, 0x73, 0x65, 0x6c, 0x65, 0x63,
+ 0x74, 0x61, 0x62, 0x6c, 0x65, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x84, 0x02,
+ 0x0a, 0x0e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+ 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18,
+ 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d,
+ 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x61, 0x62, 0x73, 0x6f, 0x6c, 0x75, 0x74, 0x65,
+ 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x41, 0x62, 0x73, 0x6f, 0x6c, 0x75,
+ 0x74, 0x65, 0x12, 0x5d, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18,
+ 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68,
+ 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70,
+ 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65,
+ 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72,
+ 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f,
+ 0x6e, 0x22, 0x51, 0x0a, 0x0d, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69,
+ 0x6f, 0x6e, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12,
+ 0x17, 0x0a, 0x0a, 0x44, 0x45, 0x53, 0x43, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0xff, 0xff,
+ 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x41, 0x53, 0x43, 0x45,
+ 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x56, 0x45, 0x52,
+ 0x53, 0x45, 0x10, 0x02, 0x22, 0xf3, 0x01, 0x0a, 0x10, 0x53, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x62,
+ 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73,
+ 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69,
+ 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+ 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63,
+ 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49,
+ 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
+ 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
+ 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
+ 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49,
+ 0x64, 0x12, 0x47, 0x0a, 0x05, 0x73, 0x6f, 0x72, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b,
+ 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
+ 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
+ 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+ 0x74, 0x6f, 0x72, 0x52, 0x05, 0x73, 0x6f, 0x72, 0x74, 0x73, 0x22, 0xf4, 0x01, 0x0a, 0x12, 0x46,
+ 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+ 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
+ 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
+ 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52,
+ 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09, 0x73, 0x6f, 0x75,
+ 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69,
+ 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+ 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63,
+ 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52,
+ 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x46, 0x0a, 0x07, 0x66, 0x69, 0x6c,
+ 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e,
0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
- 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43,
+ 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72,
+ 0x73, 0x22, 0xca, 0x02, 0x0a, 0x0e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69,
+ 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65,
+ 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63,
+ 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b,
+ 0x65, 0x74, 0x52, 0x08, 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, 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,
+ 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65,
+ 0x48, 0x00, 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x46, 0x0a,
+ 0x07, 0x6c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a,
0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72,
- 0x70, 0x63, 0x2e, 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,
+ 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, 0x43, 0x6f, 0x6e,
- 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x22,
- 0xca, 0x02, 0x0a, 0x0e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x71, 0x75, 0x65,
- 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18,
- 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68,
- 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70,
- 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74,
- 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0c, 0x73, 0x74,
- 0x61, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x6f, 0x77, 0x18, 0x02, 0x20, 0x01, 0x28, 0x12,
- 0x42, 0x02, 0x30, 0x01, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x6f,
- 0x77, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65,
- 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61,
- 0x6d, 0x65, 0x12, 0x49, 0x0a, 0x0a, 0x73, 0x65, 0x65, 0x6b, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65,
- 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
- 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b,
- 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x69, 0x74, 0x65, 0x72,
- 0x61, 0x6c, 0x52, 0x09, 0x73, 0x65, 0x65, 0x6b, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x20, 0x0a,
- 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x65, 0x18, 0x05, 0x20, 0x01,
- 0x28, 0x08, 0x52, 0x0b, 0x69, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x76, 0x65, 0x12,
- 0x1a, 0x0a, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28,
- 0x08, 0x52, 0x08, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x69,
- 0x73, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x77, 0x61, 0x72, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08,
- 0x52, 0x0a, 0x69, 0x73, 0x42, 0x61, 0x63, 0x6b, 0x77, 0x61, 0x72, 0x64, 0x22, 0x34, 0x0a, 0x0f,
- 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
- 0x21, 0x0a, 0x0a, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x52, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52,
- 0x6f, 0x77, 0x22, 0x2c, 0x0a, 0x09, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12,
- 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65,
- 0x22, 0xd0, 0x01, 0x0a, 0x07, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x12, 0x23, 0x0a, 0x0c,
- 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x09, 0x48, 0x00, 0x52, 0x0b, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75,
- 0x65, 0x12, 0x23, 0x0a, 0x0c, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75,
- 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x0b, 0x64, 0x6f, 0x75, 0x62, 0x6c,
- 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x1f, 0x0a, 0x0a, 0x62, 0x6f, 0x6f, 0x6c, 0x5f, 0x76,
- 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x09, 0x62, 0x6f,
- 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x23, 0x0a, 0x0a, 0x6c, 0x6f, 0x6e, 0x67, 0x5f,
- 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x48,
- 0x00, 0x52, 0x09, 0x6c, 0x6f, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x2c, 0x0a, 0x0f,
- 0x6e, 0x61, 0x6e, 0x6f, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
- 0x05, 0x20, 0x01, 0x28, 0x12, 0x42, 0x02, 0x30, 0x01, 0x48, 0x00, 0x52, 0x0d, 0x6e, 0x61, 0x6e,
- 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x76, 0x61,
- 0x6c, 0x75, 0x65, 0x22, 0xa5, 0x01, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x4c, 0x0a,
- 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x2c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
- 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
- 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00,
- 0x52, 0x09, 0x72, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x46, 0x0a, 0x07, 0x6c,
- 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x69,
+ 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, 0x4c, 0x69, 0x74, 0x65, 0x72, 0x61, 0x6c, 0x48, 0x00, 0x52, 0x07, 0x6c, 0x69, 0x74, 0x65,
- 0x72, 0x61, 0x6c, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x82, 0x06, 0x0a, 0x09,
- 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x03, 0x61, 0x6e, 0x64,
- 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
- 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b,
- 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6e, 0x64, 0x43, 0x6f,
- 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x03, 0x61, 0x6e, 0x64, 0x12, 0x40,
- 0x0a, 0x02, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x69, 0x6f, 0x2e,
- 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
- 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4f,
- 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x02, 0x6f, 0x72,
- 0x12, 0x43, 0x0a, 0x03, 0x6e, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e,
- 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f,
- 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70,
- 0x63, 0x2e, 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,
+ 0x2e, 0x4f, 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x02,
+ 0x6f, 0x72, 0x12, 0x43, 0x0a, 0x03, 0x6e, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70,
+ 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67,
+ 0x72, 0x70, 0x63, 0x2e, 0x4e, 0x6f, 0x74, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e,
+ 0x48, 0x00, 0x52, 0x03, 0x6e, 0x6f, 0x74, 0x12, 0x4f, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x70, 0x61,
+ 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
- 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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,
+ 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, 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, 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,
+ 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, 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,
+ 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,
- 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,
+ 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, 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,
+ 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, 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,
+ 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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, 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,
+ 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, 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,
+ 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, 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,
+ 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, 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,
+ 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,
+ 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, 0x52, 0x08, 0x73, 0x6f, 0x75,
+ 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x70, 0x69, 0x78, 0x65, 0x6c, 0x5f, 0x63,
+ 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x70, 0x69, 0x78, 0x65,
+ 0x6c, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x65, 0x0a, 0x0a, 0x7a, 0x6f, 0x6f, 0x6d, 0x5f, 0x72,
+ 0x61, 0x6e, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x52,
+ 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c,
+ 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x5a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, 0x6e,
+ 0x67, 0x65, 0x52, 0x09, 0x7a, 0x6f, 0x6f, 0x6d, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x22, 0x0a,
+ 0x0d, 0x78, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05,
+ 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x78, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d,
+ 0x65, 0x12, 0x24, 0x0a, 0x0e, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61,
+ 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x79, 0x43, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x1a, 0x8f, 0x01, 0x0a, 0x09, 0x5a, 0x6f, 0x6f, 0x6d,
+ 0x52, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x2d, 0x0a, 0x0e, 0x6d, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74,
+ 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x30,
+ 0x01, 0x48, 0x00, 0x52, 0x0c, 0x6d, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6e, 0x6f,
+ 0x73, 0x88, 0x01, 0x01, 0x12, 0x2d, 0x0a, 0x0e, 0x6d, 0x61, 0x78, 0x5f, 0x64, 0x61, 0x74, 0x65,
+ 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x30, 0x01,
+ 0x48, 0x01, 0x52, 0x0c, 0x6d, 0x61, 0x78, 0x44, 0x61, 0x74, 0x65, 0x4e, 0x61, 0x6e, 0x6f, 0x73,
+ 0x88, 0x01, 0x01, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x69, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x65,
+ 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x42, 0x11, 0x0a, 0x0f, 0x5f, 0x6d, 0x61, 0x78, 0x5f, 0x64,
+ 0x61, 0x74, 0x65, 0x5f, 0x6e, 0x61, 0x6e, 0x6f, 0x73, 0x22, 0xc3, 0x06, 0x0a, 0x17, 0x43, 0x72,
+ 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f,
0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63,
- 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 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,
+ 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, 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,
+ 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, 0x54, 0x69, 0x63, 0x6b, 0x65,
- 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x5b, 0x0a, 0x0f, 0x73,
- 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
- 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
- 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
- 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63,
- 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65,
- 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x48, 0x00, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65,
- 0x6d, 0x61, 0x12, 0x5d, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b,
- 0x32, 0x49, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
- 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
- 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74,
- 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70,
- 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, 0x6b, 0x69, 0x6e,
- 0x64, 0x1a, 0xfb, 0x03, 0x0a, 0x0e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x91, 0x01, 0x0a, 0x15, 0x69, 0x6e, 0x5f, 0x6d, 0x65, 0x6d, 0x6f,
- 0x72, 0x79, 0x5f, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x01,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5c, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
- 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
- 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49,
- 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69, 0x6e, 0x64, 0x2e,
- 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e,
- 0x6c, 0x79, 0x48, 0x00, 0x52, 0x12, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70,
- 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x8e, 0x01, 0x0a, 0x14, 0x69, 0x6e, 0x5f,
- 0x6d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x62, 0x61, 0x63, 0x6b, 0x65,
- 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x5b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65,
- 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63,
- 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65, 0x61,
- 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b, 0x69,
- 0x6e, 0x64, 0x2e, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x42, 0x61,
- 0x63, 0x6b, 0x65, 0x64, 0x48, 0x00, 0x52, 0x11, 0x69, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79,
- 0x4b, 0x65, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x67, 0x0a, 0x05, 0x62, 0x6c, 0x69,
- 0x6e, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
- 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
- 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x65,
- 0x61, 0x74, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x4b,
- 0x69, 0x6e, 0x64, 0x2e, 0x42, 0x6c, 0x69, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x05, 0x62, 0x6c, 0x69,
- 0x6e, 0x6b, 0x1a, 0x14, 0x0a, 0x12, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x41, 0x70,
- 0x70, 0x65, 0x6e, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x1a, 0x34, 0x0a, 0x11, 0x49, 0x6e, 0x4d, 0x65,
- 0x6d, 0x6f, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x42, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x12, 0x1f, 0x0a,
- 0x0b, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03,
- 0x28, 0x09, 0x52, 0x0a, 0x6b, 0x65, 0x79, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x1a, 0x07,
- 0x0a, 0x05, 0x42, 0x6c, 0x69, 0x6e, 0x6b, 0x42, 0x06, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x42,
- 0x0c, 0x0a, 0x0a, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xb8, 0x02,
- 0x0a, 0x0e, 0x57, 0x68, 0x65, 0x72, 0x65, 0x49, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
- 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
- 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61,
- 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08,
- 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4a, 0x0a, 0x07, 0x6c, 0x65, 0x66, 0x74,
- 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 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, 0x54, 0x61,
- 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x06, 0x6c, 0x65,
- 0x66, 0x74, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f, 0x69, 0x64,
- 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
- 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b,
- 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x72, 0x69, 0x67, 0x68, 0x74,
- 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64, 0x18, 0x04,
- 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64, 0x12, 0x28,
- 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d, 0x61, 0x74,
- 0x63, 0x68, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e,
- 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x9c, 0x02, 0x0a, 0x17, 0x43, 0x6f, 0x6c,
- 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71,
- 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x5f, 0x69,
- 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65,
- 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63,
- 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b,
- 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x09,
- 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
- 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70,
- 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67,
- 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e,
- 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b,
- 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28,
- 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x31, 0x0a,
- 0x12, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x6c, 0x69,
- 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x10, 0x75, 0x6e, 0x69,
- 0x71, 0x75, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x88, 0x01, 0x01,
- 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75,
- 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0xa0, 0x02, 0x0a, 0x0c, 0x53, 0x6c, 0x69, 0x63,
- 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75,
- 0x6c, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f,
+ 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, 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,
+ 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x06,
+ 0x6c, 0x65, 0x66, 0x74, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x08, 0x72, 0x69, 0x67, 0x68, 0x74, 0x5f,
+ 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
+ 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
+ 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62,
+ 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x72, 0x69, 0x67,
+ 0x68, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64,
+ 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x65, 0x64,
+ 0x12, 0x28, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6d,
+ 0x61, 0x74, 0x63, 0x68, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x75,
+ 0x6d, 0x6e, 0x73, 0x54, 0x6f, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x22, 0x9c, 0x02, 0x0a, 0x17, 0x43,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52,
+ 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74,
+ 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62,
+ 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69,
+ 0x63, 0x6b, 0x65, 0x74, 0x52, 0x08, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x49, 0x64, 0x12, 0x4e,
+ 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28,
+ 0x0b, 0x32, 0x31, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e,
+ 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65,
+ 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72,
+ 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x64, 0x12, 0x1f,
+ 0x0a, 0x0b, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20,
+ 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12,
+ 0x31, 0x0a, 0x12, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f,
+ 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x10, 0x75,
+ 0x6e, 0x69, 0x71, 0x75, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x88,
+ 0x01, 0x01, 0x42, 0x15, 0x0a, 0x13, 0x5f, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x76, 0x61,
+ 0x6c, 0x75, 0x65, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x22, 0xa0, 0x02, 0x0a, 0x0c, 0x53, 0x6c,
+ 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x46, 0x0a, 0x09, 0x72, 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, 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,
+ 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, 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,
+ 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, 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, 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,
+ 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, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
+ 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, 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,
+ 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, 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,
+ 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54,
+ 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x66,
+ 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x74, 0x0a, 0x13, 0x75, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63,
+ 0x74, 0x75, 0x72, 0x65, 0x64, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01,
+ 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65,
0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e,
- 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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,
+ 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75,
+ 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x12, 0x75, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63,
+ 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x04, 0x73,
+ 0x6f, 0x72, 0x74, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62,
+ 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f,
+ 0x72, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00,
+ 0x52, 0x04, 0x73, 0x6f, 0x72, 0x74, 0x12, 0x4a, 0x0a, 0x04, 0x68, 0x65, 0x61, 0x64, 0x18, 0x0d,
+ 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
+ 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
+ 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54,
+ 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x04, 0x68, 0x65,
+ 0x61, 0x64, 0x12, 0x4a, 0x0a, 0x04, 0x74, 0x61, 0x69, 0x6c, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b,
+ 0x32, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
- 0x67, 0x72, 0x70, 0x63, 0x2e, 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,
+ 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, 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,
+ 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, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44,
- 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x48, 0x00, 0x52, 0x12, 0x72, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72, 0x74, 0x44, 0x6f, 0x77, 0x6e,
- 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0a, 0x63, 0x72, 0x6f, 0x73, 0x73, 0x5f,
- 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e,
- 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
- 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43,
- 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x63, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f,
- 0x69, 0x6e, 0x12, 0x60, 0x0a, 0x0c, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x5f, 0x6a, 0x6f,
- 0x69, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
- 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
- 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4e, 0x61, 0x74,
- 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65,
- 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c,
- 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5a, 0x0a, 0x0a, 0x65, 0x78, 0x61, 0x63, 0x74, 0x5f, 0x6a, 0x6f,
- 0x69, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
+ 0x65, 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, 0x45, 0x78, 0x61,
- 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e,
- 0x12, 0x57, 0x0a, 0x09, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x1a, 0x20,
- 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
- 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61,
- 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e,
- 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52,
- 0x08, 0x6c, 0x65, 0x66, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5c, 0x0a, 0x0a, 0x61, 0x73, 0x5f,
- 0x6f, 0x66, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e,
- 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f,
- 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70,
- 0x63, 0x2e, 0x41, 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73,
- 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x08, 0x61,
- 0x73, 0x4f, 0x66, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x57, 0x0a, 0x0b, 0x66, 0x65, 0x74, 0x63, 0x68,
- 0x5f, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x69,
+ 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, 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,
+ 0x2e, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73,
+ 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x63, 0x72, 0x6f, 0x73, 0x73,
+ 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x60, 0x0a, 0x0c, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x5f,
+ 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4e,
+ 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73,
+ 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0b, 0x6e, 0x61, 0x74, 0x75, 0x72,
+ 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x5a, 0x0a, 0x0a, 0x65, 0x78, 0x61, 0x63, 0x74, 0x5f,
+ 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45,
+ 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x65, 0x78, 0x61, 0x63, 0x74, 0x4a, 0x6f,
+ 0x69, 0x6e, 0x12, 0x57, 0x0a, 0x09, 0x6c, 0x65, 0x66, 0x74, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18,
+ 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68,
+ 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70,
+ 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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, 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,
+ 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, 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, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
+ 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, 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,
+ 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, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67,
- 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x47, 0x0a, 0x02, 0x61, 0x6a, 0x18, 0x28, 0x20, 0x01, 0x28,
- 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e,
- 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65,
- 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x02, 0x61, 0x6a, 0x12, 0x49,
- 0x0a, 0x03, 0x72, 0x61, 0x6a, 0x18, 0x29, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f,
- 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
- 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e,
- 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
- 0x73, 0x74, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x6a, 0x12, 0x69, 0x0a, 0x11, 0x63, 0x6f, 0x6c,
- 0x75, 0x6d, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x18, 0x2a,
- 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
- 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
- 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53,
- 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x48, 0x00, 0x52, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73,
- 0x74, 0x69, 0x63, 0x73, 0x12, 0x5a, 0x0a, 0x0a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x6a, 0x6f,
- 0x69, 0x6e, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
- 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
- 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x75, 0x6c,
- 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75,
- 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x4a, 0x6f, 0x69, 0x6e,
- 0x12, 0x47, 0x0a, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, 0x32,
- 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70,
- 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67,
- 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
- 0x48, 0x00, 0x52, 0x05, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x42, 0x04, 0x0a, 0x02, 0x6f, 0x70, 0x4a,
- 0x04, 0x08, 0x14, 0x10, 0x15, 0x4a, 0x04, 0x08, 0x1d, 0x10, 0x1e, 0x2a, 0x62, 0x0a, 0x0f, 0x42,
- 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x23,
- 0x0a, 0x1f, 0x42, 0x41, 0x44, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56,
- 0x49, 0x4f, 0x52, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45,
- 0x44, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x54, 0x48, 0x52, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x09,
- 0x0a, 0x05, 0x52, 0x45, 0x53, 0x45, 0x54, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x53, 0x4b, 0x49,
- 0x50, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x50, 0x4f, 0x49, 0x53, 0x4f, 0x4e, 0x10, 0x04, 0x2a,
- 0x74, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x42, 0x79, 0x4e, 0x75, 0x6c, 0x6c, 0x42,
- 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x1f, 0x0a, 0x1b, 0x4e, 0x55, 0x4c, 0x4c, 0x5f,
- 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x53, 0x50, 0x45,
- 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4e, 0x55, 0x4c, 0x4c,
- 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x53, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f,
- 0x56, 0x41, 0x4c, 0x55, 0x45, 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x53, 0x10,
- 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x5a, 0x45, 0x52, 0x4f, 0x5f, 0x44, 0x4f, 0x4d, 0x49, 0x4e, 0x41,
- 0x54, 0x45, 0x53, 0x10, 0x03, 0x2a, 0x1b, 0x0a, 0x09, 0x4e, 0x75, 0x6c, 0x6c, 0x56, 0x61, 0x6c,
- 0x75, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x55, 0x4c, 0x4c, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45,
- 0x10, 0x00, 0x2a, 0x32, 0x0a, 0x0f, 0x43, 0x61, 0x73, 0x65, 0x53, 0x65, 0x6e, 0x73, 0x69, 0x74,
- 0x69, 0x76, 0x69, 0x74, 0x79, 0x12, 0x0e, 0x0a, 0x0a, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x5f, 0x43,
- 0x41, 0x53, 0x45, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x49, 0x47, 0x4e, 0x4f, 0x52, 0x45, 0x5f,
- 0x43, 0x41, 0x53, 0x45, 0x10, 0x01, 0x2a, 0x26, 0x0a, 0x09, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54,
- 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x47, 0x55, 0x4c, 0x41, 0x52, 0x10, 0x00,
- 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x4e, 0x56, 0x45, 0x52, 0x54, 0x45, 0x44, 0x10, 0x01, 0x32, 0xb9,
- 0x32, 0x0a, 0x0c, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12,
- 0x91, 0x01, 0x0a, 0x20, 0x47, 0x65, 0x74, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54,
- 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61,
- 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c,
- 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x63, 0x6b, 0x65, 0x74, 0x1a,
- 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70,
- 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67,
- 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c,
- 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x0a, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62,
- 0x6c, 0x65, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65,
- 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e,
- 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x46, 0x65, 0x74, 0x63, 0x68, 0x54, 0x61, 0x62, 0x6c,
- 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
+ 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, 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, 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,
+ 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, 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,
+ 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f,
+ 0x74, 0x12, 0x62, 0x0a, 0x0d, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x5f, 0x77, 0x68,
+ 0x65, 0x6e, 0x18, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
+ 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
+ 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61,
+ 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0c, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f,
+ 0x74, 0x57, 0x68, 0x65, 0x6e, 0x12, 0x54, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, 0x74, 0x61,
+ 0x62, 0x6c, 0x65, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62,
+ 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65,
+ 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00,
+ 0x52, 0x09, 0x6d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x5a, 0x0a, 0x0a, 0x72,
+ 0x61, 0x6e, 0x67, 0x65, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70,
+ 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67,
+ 0x72, 0x70, 0x63, 0x2e, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62,
+ 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x09, 0x72, 0x61,
+ 0x6e, 0x67, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x12, 0x47, 0x0a, 0x02, 0x61, 0x6a, 0x18, 0x28, 0x20,
+ 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61,
- 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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,
+ 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, 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,
+ 0x63, 0x2e, 0x41, 0x6a, 0x52, 0x61, 0x6a, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71,
+ 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x03, 0x72, 0x61, 0x6a, 0x12, 0x69, 0x0a, 0x11, 0x63,
+ 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73,
+ 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
+ 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b,
+ 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d,
+ 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
+ 0x73, 0x74, 0x48, 0x00, 0x52, 0x10, 0x63, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74,
+ 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x5a, 0x0a, 0x0a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f,
+ 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x69, 0x6f, 0x2e,
0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
- 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e,
+ 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, 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,
+ 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, 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,
+ 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, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74,
+ 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x86, 0x01, 0x0a, 0x0a, 0x45, 0x6d,
+ 0x70, 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x34, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
+ 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
+ 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x6d, 0x70,
+ 0x74, 0x79, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40,
+ 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72,
+ 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72,
+ 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65,
+ 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x09, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65,
+ 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
+ 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
+ 0x67, 0x72, 0x70, 0x63, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68,
+ 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70,
+ 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74,
+ 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,
+ 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x44, 0x72,
+ 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62,
+ 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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, 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,
+ 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, 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,
+ 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, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
+ 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, 0x55, 0x6e, 0x73,
- 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x54,
- 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f,
- 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
- 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e,
- 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65,
- 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
- 0x7f, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65,
- 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63,
- 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f, 0x72, 0x74,
- 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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,
+ 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, 0x48, 0x65,
- 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
+ 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, 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,
+ 0x65, 0x22, 0x00, 0x12, 0x9b, 0x01, 0x0a, 0x12, 0x55, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74,
+ 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x41, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x55,
+ 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x75, 0x72, 0x65, 0x64, 0x46, 0x69, 0x6c, 0x74, 0x65,
+ 0x72, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e,
+ 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f,
+ 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70,
+ 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43,
+ 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x00, 0x12, 0x7f, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62,
+ 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6f,
+ 0x72, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f,
+ 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+ 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e,
+ 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65,
+ 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
+ 0x84, 0x01, 0x0a, 0x06, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x12, 0x36, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x48,
+ 0x65, 0x61, 0x64, 0x4f, 0x72, 0x54, 0x61, 0x69, 0x6c, 0x42, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65,
0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e,
0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54,
0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70,
- 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 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,
+ 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, 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,
+ 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, 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,
+ 0x73, 0x65, 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f,
+ 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
+ 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61,
+ 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x6f,
+ 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76,
0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61,
0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64,
0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73,
- 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x0b, 0x4d, 0x65, 0x72, 0x67,
- 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x35, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65,
- 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63,
- 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65, 0x72, 0x67,
- 0x65, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40,
- 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72,
- 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72,
- 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
- 0x22, 0x00, 0x12, 0x90, 0x01, 0x0a, 0x0f, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x4a, 0x6f, 0x69, 0x6e,
- 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x39, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70,
- 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b,
- 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x43, 0x72, 0x6f, 0x73, 0x73,
- 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
- 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e,
- 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65,
- 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61,
- 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f,
- 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x94, 0x01, 0x0a, 0x11, 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61,
- 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x2e, 0x69, 0x6f,
- 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
- 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e,
- 0x4e, 0x61, 0x74, 0x75, 0x72, 0x61, 0x6c, 0x4a, 0x6f, 0x69, 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65,
- 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65,
- 0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 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, 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,
+ 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,
- 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,
+ 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, 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,
+ 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, 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,
+ 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, 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, 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, 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,
+ 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, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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,
- 0x80, 0x01, 0x0a, 0x07, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x12, 0x31, 0x2e, 0x69, 0x6f,
+ 0x8f, 0x01, 0x0a, 0x0c, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65, 0x6e,
+ 0x12, 0x3b, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
+ 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
+ 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x57, 0x68, 0x65,
+ 0x6e, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e,
+ 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f,
+ 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70,
+ 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43,
+ 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+ 0x00, 0x12, 0x80, 0x01, 0x0a, 0x07, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x12, 0x31, 0x2e,
+ 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f,
+ 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70,
+ 0x63, 0x2e, 0x46, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+ 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e,
+ 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e,
+ 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62,
+ 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+ 0x73, 0x65, 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x12, 0x52, 0x75, 0x6e, 0x43, 0x68, 0x61, 0x72,
+ 0x74, 0x44, 0x6f, 0x77, 0x6e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x12, 0x3c, 0x2e, 0x69, 0x6f,
0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 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, 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,
+ 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, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67,
- 0x72, 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c,
- 0x65, 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
- 0x65, 0x22, 0x00, 0x12, 0x83, 0x01, 0x0a, 0x05, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x34, 0x2e,
- 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 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,
+ 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, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 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,
+ 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f, 0x77, 0x52, 0x65,
+ 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68,
+ 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70,
+ 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x65, 0x65, 0x6b, 0x52, 0x6f,
+ 0x77, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x84, 0x01, 0x0a, 0x09,
+ 0x4d, 0x65, 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x12, 0x33, 0x2e, 0x69, 0x6f, 0x2e, 0x64,
+ 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62,
+ 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x4d, 0x65,
+ 0x74, 0x61, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40,
+ 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72,
+ 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72,
+ 0x70, 0x63, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65,
+ 0x43, 0x72, 0x65, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+ 0x22, 0x00, 0x12, 0x99, 0x01, 0x0a, 0x17, 0x43, 0x6f, 0x6d, 0x70, 0x75, 0x74, 0x65, 0x43, 0x6f,
+ 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x3a,
+ 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72,
+ 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72,
+ 0x70, 0x63, 0x2e, 0x43, 0x6f, 0x6c, 0x75, 0x6d, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74,
+ 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x40, 0x2e, 0x69, 0x6f, 0x2e,
+ 0x64, 0x65, 0x65, 0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
+ 0x62, 0x61, 0x63, 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x45,
+ 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x43, 0x72, 0x65, 0x61,
+ 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7c,
+ 0x0a, 0x05, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x12, 0x2f, 0x2e, 0x69, 0x6f, 0x2e, 0x64, 0x65, 0x65,
0x70, 0x68, 0x61, 0x76, 0x65, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x62, 0x61, 0x63,
- 0x6b, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 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,
+ 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, 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,
+ 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 (
@@ -12749,7 +12959,7 @@ func file_deephaven_core_proto_table_proto_rawDescGZIP() []byte {
}
var file_deephaven_core_proto_table_proto_enumTypes = make([]protoimpl.EnumInfo, 12)
-var file_deephaven_core_proto_table_proto_msgTypes = make([]protoimpl.MessageInfo, 128)
+var file_deephaven_core_proto_table_proto_msgTypes = make([]protoimpl.MessageInfo, 130)
var file_deephaven_core_proto_table_proto_goTypes = []interface{}{
(BadDataBehavior)(0), // 0: io.deephaven.proto.backplane.grpc.BadDataBehavior
(UpdateByNullBehavior)(0), // 1: io.deephaven.proto.backplane.grpc.UpdateByNullBehavior
@@ -12833,77 +13043,79 @@ var file_deephaven_core_proto_table_proto_goTypes = []interface{}{
(*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_AggregationCountWhere)(nil), // 130: io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere
- (*Aggregation_AggregationRowKey)(nil), // 131: io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey
- (*Aggregation_AggregationPartition)(nil), // 132: io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition
- (*Aggregation_AggregationFormula)(nil), // 133: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula
- (*RunChartDownsampleRequest_ZoomRange)(nil), // 134: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange
- (*CreateInputTableRequest_InputTableKind)(nil), // 135: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind
- (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly)(nil), // 136: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly
- (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked)(nil), // 137: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked
- (*CreateInputTableRequest_InputTableKind_Blink)(nil), // 138: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink
- (*BatchTableRequest_Operation)(nil), // 139: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation
- (*ticket.Ticket)(nil), // 140: io.deephaven.proto.backplane.grpc.Ticket
+ (*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
+ (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere)(nil), // 104: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountWhere
+ (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere)(nil), // 105: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeCountWhere
+ (*ComboAggregateRequest_Aggregate)(nil), // 106: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate
+ (*AggSpec_AggSpecApproximatePercentile)(nil), // 107: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentile
+ (*AggSpec_AggSpecCountDistinct)(nil), // 108: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinct
+ (*AggSpec_AggSpecDistinct)(nil), // 109: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinct
+ (*AggSpec_AggSpecFormula)(nil), // 110: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormula
+ (*AggSpec_AggSpecMedian)(nil), // 111: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedian
+ (*AggSpec_AggSpecPercentile)(nil), // 112: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentile
+ (*AggSpec_AggSpecSorted)(nil), // 113: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted
+ (*AggSpec_AggSpecSortedColumn)(nil), // 114: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn
+ (*AggSpec_AggSpecTDigest)(nil), // 115: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigest
+ (*AggSpec_AggSpecUnique)(nil), // 116: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique
+ (*AggSpec_AggSpecNonUniqueSentinel)(nil), // 117: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel
+ (*AggSpec_AggSpecWeighted)(nil), // 118: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted
+ (*AggSpec_AggSpecAbsSum)(nil), // 119: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum
+ (*AggSpec_AggSpecAvg)(nil), // 120: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvg
+ (*AggSpec_AggSpecFirst)(nil), // 121: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirst
+ (*AggSpec_AggSpecFreeze)(nil), // 122: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreeze
+ (*AggSpec_AggSpecGroup)(nil), // 123: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroup
+ (*AggSpec_AggSpecLast)(nil), // 124: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLast
+ (*AggSpec_AggSpecMax)(nil), // 125: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMax
+ (*AggSpec_AggSpecMin)(nil), // 126: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMin
+ (*AggSpec_AggSpecStd)(nil), // 127: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStd
+ (*AggSpec_AggSpecSum)(nil), // 128: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSum
+ (*AggSpec_AggSpecVar)(nil), // 129: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVar
+ (*Aggregation_AggregationColumns)(nil), // 130: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns
+ (*Aggregation_AggregationCount)(nil), // 131: io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount
+ (*Aggregation_AggregationCountWhere)(nil), // 132: io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere
+ (*Aggregation_AggregationRowKey)(nil), // 133: io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey
+ (*Aggregation_AggregationPartition)(nil), // 134: io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition
+ (*Aggregation_AggregationFormula)(nil), // 135: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula
+ (*RunChartDownsampleRequest_ZoomRange)(nil), // 136: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange
+ (*CreateInputTableRequest_InputTableKind)(nil), // 137: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind
+ (*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly)(nil), // 138: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly
+ (*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked)(nil), // 139: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked
+ (*CreateInputTableRequest_InputTableKind_Blink)(nil), // 140: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink
+ (*BatchTableRequest_Operation)(nil), // 141: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation
+ (*ticket.Ticket)(nil), // 142: io.deephaven.proto.backplane.grpc.Ticket
}
var file_deephaven_core_proto_table_proto_depIdxs = []int32{
- 140, // 0: io.deephaven.proto.backplane.grpc.TableReference.ticket:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 3: io.deephaven.proto.backplane.grpc.FetchTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 5: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
- 140, // 6: io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage.export_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
- 140, // 7: io.deephaven.proto.backplane.grpc.EmptyTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
- 140, // 8: io.deephaven.proto.backplane.grpc.TimeTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
- 140, // 9: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 5: io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 6: io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage.export_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 7: io.deephaven.proto.backplane.grpc.EmptyTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 8: io.deephaven.proto.backplane.grpc.TimeTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 9: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 10: io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
5, // 11: io.deephaven.proto.backplane.grpc.MathContext.rounding_mode:type_name -> io.deephaven.proto.backplane.grpc.MathContext.RoundingMode
77, // 12: io.deephaven.proto.backplane.grpc.UpdateByWindowScale.ticks:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicks
@@ -12915,105 +13127,105 @@ var file_deephaven_core_proto_table_proto_depIdxs = []int32{
0, // 18: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.on_zero_delta_time:type_name -> io.deephaven.proto.backplane.grpc.BadDataBehavior
22, // 19: io.deephaven.proto.backplane.grpc.UpdateByEmOptions.big_value_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext
1, // 20: io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions.null_behavior:type_name -> io.deephaven.proto.backplane.grpc.UpdateByNullBehavior
- 140, // 21: io.deephaven.proto.backplane.grpc.UpdateByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 21: io.deephaven.proto.backplane.grpc.UpdateByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 22: io.deephaven.proto.backplane.grpc.UpdateByRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
79, // 23: io.deephaven.proto.backplane.grpc.UpdateByRequest.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions
80, // 24: io.deephaven.proto.backplane.grpc.UpdateByRequest.operations:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation
- 140, // 25: io.deephaven.proto.backplane.grpc.SelectDistinctRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 27: io.deephaven.proto.backplane.grpc.DropColumnsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 29: io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 31: io.deephaven.proto.backplane.grpc.HeadOrTailRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 33: io.deephaven.proto.backplane.grpc.HeadOrTailByRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 35: io.deephaven.proto.backplane.grpc.UngroupRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 37: io.deephaven.proto.backplane.grpc.MergeTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 39: io.deephaven.proto.backplane.grpc.SnapshotTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 41: io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 44: io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 47: io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 50: io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 53: io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 56: io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 60: io.deephaven.proto.backplane.grpc.AjRajTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 64: io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 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
- 140, // 66: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 66: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 67: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
12, // 68: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
7, // 69: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.range_start_rule:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeStartRule
8, // 70: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.range_end_rule:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeEndRule
49, // 71: io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.aggregations:type_name -> io.deephaven.proto.backplane.grpc.Aggregation
- 140, // 72: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 72: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 73: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 104, // 74: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.aggregates:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate
- 140, // 75: io.deephaven.proto.backplane.grpc.AggregateAllRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 106, // 74: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.aggregates:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate
+ 142, // 75: io.deephaven.proto.backplane.grpc.AggregateAllRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 76: io.deephaven.proto.backplane.grpc.AggregateAllRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
47, // 77: io.deephaven.proto.backplane.grpc.AggregateAllRequest.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec
- 117, // 78: io.deephaven.proto.backplane.grpc.AggSpec.abs_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum
- 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
- 140, // 101: io.deephaven.proto.backplane.grpc.AggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 119, // 78: io.deephaven.proto.backplane.grpc.AggSpec.abs_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSum
+ 107, // 79: io.deephaven.proto.backplane.grpc.AggSpec.approximate_percentile:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentile
+ 120, // 80: io.deephaven.proto.backplane.grpc.AggSpec.avg:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvg
+ 108, // 81: io.deephaven.proto.backplane.grpc.AggSpec.count_distinct:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinct
+ 109, // 82: io.deephaven.proto.backplane.grpc.AggSpec.distinct:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinct
+ 121, // 83: io.deephaven.proto.backplane.grpc.AggSpec.first:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirst
+ 110, // 84: io.deephaven.proto.backplane.grpc.AggSpec.formula:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormula
+ 122, // 85: io.deephaven.proto.backplane.grpc.AggSpec.freeze:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreeze
+ 123, // 86: io.deephaven.proto.backplane.grpc.AggSpec.group:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroup
+ 124, // 87: io.deephaven.proto.backplane.grpc.AggSpec.last:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLast
+ 125, // 88: io.deephaven.proto.backplane.grpc.AggSpec.max:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMax
+ 111, // 89: io.deephaven.proto.backplane.grpc.AggSpec.median:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedian
+ 126, // 90: io.deephaven.proto.backplane.grpc.AggSpec.min:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMin
+ 112, // 91: io.deephaven.proto.backplane.grpc.AggSpec.percentile:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentile
+ 113, // 92: io.deephaven.proto.backplane.grpc.AggSpec.sorted_first:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted
+ 113, // 93: io.deephaven.proto.backplane.grpc.AggSpec.sorted_last:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted
+ 127, // 94: io.deephaven.proto.backplane.grpc.AggSpec.std:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStd
+ 128, // 95: io.deephaven.proto.backplane.grpc.AggSpec.sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSum
+ 115, // 96: io.deephaven.proto.backplane.grpc.AggSpec.t_digest:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigest
+ 116, // 97: io.deephaven.proto.backplane.grpc.AggSpec.unique:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique
+ 118, // 98: io.deephaven.proto.backplane.grpc.AggSpec.weighted_avg:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted
+ 118, // 99: io.deephaven.proto.backplane.grpc.AggSpec.weighted_sum:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeighted
+ 129, // 100: io.deephaven.proto.backplane.grpc.AggSpec.var:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVar
+ 142, // 101: io.deephaven.proto.backplane.grpc.AggregateRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 102: io.deephaven.proto.backplane.grpc.AggregateRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
12, // 103: io.deephaven.proto.backplane.grpc.AggregateRequest.initial_groups_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
49, // 104: io.deephaven.proto.backplane.grpc.AggregateRequest.aggregations:type_name -> io.deephaven.proto.backplane.grpc.Aggregation
- 128, // 105: io.deephaven.proto.backplane.grpc.Aggregation.columns:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns
- 129, // 106: io.deephaven.proto.backplane.grpc.Aggregation.count:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount
- 131, // 107: io.deephaven.proto.backplane.grpc.Aggregation.first_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey
- 131, // 108: io.deephaven.proto.backplane.grpc.Aggregation.last_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey
- 132, // 109: io.deephaven.proto.backplane.grpc.Aggregation.partition:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition
- 133, // 110: io.deephaven.proto.backplane.grpc.Aggregation.formula:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula
- 130, // 111: io.deephaven.proto.backplane.grpc.Aggregation.count_where:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere
+ 130, // 105: io.deephaven.proto.backplane.grpc.Aggregation.columns:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns
+ 131, // 106: io.deephaven.proto.backplane.grpc.Aggregation.count:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCount
+ 133, // 107: io.deephaven.proto.backplane.grpc.Aggregation.first_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey
+ 133, // 108: io.deephaven.proto.backplane.grpc.Aggregation.last_row_key:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKey
+ 134, // 109: io.deephaven.proto.backplane.grpc.Aggregation.partition:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartition
+ 135, // 110: io.deephaven.proto.backplane.grpc.Aggregation.formula:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula
+ 132, // 111: io.deephaven.proto.backplane.grpc.Aggregation.count_where:type_name -> io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhere
10, // 112: io.deephaven.proto.backplane.grpc.SortDescriptor.direction:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection
- 140, // 113: io.deephaven.proto.backplane.grpc.SortTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 113: io.deephaven.proto.backplane.grpc.SortTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 114: io.deephaven.proto.backplane.grpc.SortTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
50, // 115: io.deephaven.proto.backplane.grpc.SortTableRequest.sorts:type_name -> io.deephaven.proto.backplane.grpc.SortDescriptor
- 140, // 116: io.deephaven.proto.backplane.grpc.FilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 116: io.deephaven.proto.backplane.grpc.FilterTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 117: io.deephaven.proto.backplane.grpc.FilterTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
58, // 118: io.deephaven.proto.backplane.grpc.FilterTableRequest.filters:type_name -> io.deephaven.proto.backplane.grpc.Condition
- 140, // 119: io.deephaven.proto.backplane.grpc.SeekRowRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 119: io.deephaven.proto.backplane.grpc.SeekRowRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
56, // 120: io.deephaven.proto.backplane.grpc.SeekRowRequest.seek_value:type_name -> io.deephaven.proto.backplane.grpc.Literal
55, // 121: io.deephaven.proto.backplane.grpc.Value.reference:type_name -> io.deephaven.proto.backplane.grpc.Reference
56, // 122: io.deephaven.proto.backplane.grpc.Value.literal:type_name -> io.deephaven.proto.backplane.grpc.Literal
@@ -13048,24 +13260,24 @@ var file_deephaven_core_proto_table_proto_depIdxs = []int32{
3, // 151: io.deephaven.proto.backplane.grpc.ContainsCondition.case_sensitivity:type_name -> io.deephaven.proto.backplane.grpc.CaseSensitivity
4, // 152: io.deephaven.proto.backplane.grpc.ContainsCondition.match_type:type_name -> io.deephaven.proto.backplane.grpc.MatchType
55, // 153: io.deephaven.proto.backplane.grpc.SearchCondition.optional_references:type_name -> io.deephaven.proto.backplane.grpc.Reference
- 140, // 154: io.deephaven.proto.backplane.grpc.FlattenRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 154: io.deephaven.proto.backplane.grpc.FlattenRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 155: io.deephaven.proto.backplane.grpc.FlattenRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 140, // 156: io.deephaven.proto.backplane.grpc.MetaTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 156: io.deephaven.proto.backplane.grpc.MetaTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 157: io.deephaven.proto.backplane.grpc.MetaTableRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 140, // 158: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 158: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 159: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 134, // 160: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.zoom_range:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange
- 140, // 161: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 136, // 160: io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.zoom_range:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange
+ 142, // 161: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 162: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.source_table_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 135, // 163: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.kind:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind
- 140, // 164: io.deephaven.proto.backplane.grpc.WhereInRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 137, // 163: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.kind:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind
+ 142, // 164: io.deephaven.proto.backplane.grpc.WhereInRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 165: io.deephaven.proto.backplane.grpc.WhereInRequest.left_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
12, // 166: io.deephaven.proto.backplane.grpc.WhereInRequest.right_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 140, // 167: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 167: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 168: io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 140, // 169: io.deephaven.proto.backplane.grpc.SliceRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
+ 142, // 169: io.deephaven.proto.backplane.grpc.SliceRequest.result_id:type_name -> io.deephaven.proto.backplane.grpc.Ticket
12, // 170: io.deephaven.proto.backplane.grpc.SliceRequest.source_id:type_name -> io.deephaven.proto.backplane.grpc.TableReference
- 139, // 171: io.deephaven.proto.backplane.grpc.BatchTableRequest.ops:type_name -> io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation
+ 141, // 171: io.deephaven.proto.backplane.grpc.BatchTableRequest.ops:type_name -> io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation
22, // 172: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions.math_context:type_name -> io.deephaven.proto.backplane.grpc.MathContext
81, // 173: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.column:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn
82, // 174: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.spec:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec
@@ -13090,185 +13302,189 @@ var file_deephaven_core_proto_table_proto_depIdxs = []int32{
101, // 193: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_std:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd
102, // 194: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_wavg:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg
103, // 195: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_formula:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula
- 24, // 196: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
- 23, // 197: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 24, // 198: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
- 23, // 199: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 24, // 200: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
- 23, // 201: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 24, // 202: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
- 23, // 203: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 24, // 204: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
- 23, // 205: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 25, // 206: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions
- 23, // 207: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 208: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 209: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 210: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 211: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 212: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 213: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 214: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 215: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 216: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 217: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 218: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 219: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 220: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 221: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 222: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 223: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 224: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 225: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 23, // 226: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
- 9, // 227: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate.type:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType
- 112, // 228: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted.columns:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn
- 115, // 229: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique.non_unique_sentinel:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel
- 2, // 230: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel.null_value:type_name -> io.deephaven.proto.backplane.grpc.NullValue
- 47, // 231: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec
- 21, // 232: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula.selectable:type_name -> io.deephaven.proto.backplane.grpc.Selectable
- 136, // 233: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_append_only:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly
- 137, // 234: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_key_backed:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked
- 138, // 235: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.blink:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink
- 18, // 236: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.empty_table:type_name -> io.deephaven.proto.backplane.grpc.EmptyTableRequest
- 19, // 237: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.time_table:type_name -> io.deephaven.proto.backplane.grpc.TimeTableRequest
- 28, // 238: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.drop_columns:type_name -> io.deephaven.proto.backplane.grpc.DropColumnsRequest
- 20, // 239: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 240: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.lazy_update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 241: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 242: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 243: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 27, // 244: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select_distinct:type_name -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest
- 52, // 245: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.filter:type_name -> io.deephaven.proto.backplane.grpc.FilterTableRequest
- 29, // 246: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.unstructured_filter:type_name -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest
- 51, // 247: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.sort:type_name -> io.deephaven.proto.backplane.grpc.SortTableRequest
- 30, // 248: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
- 30, // 249: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
- 31, // 250: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
- 31, // 251: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
- 32, // 252: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.ungroup:type_name -> io.deephaven.proto.backplane.grpc.UngroupRequest
- 33, // 253: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.merge:type_name -> io.deephaven.proto.backplane.grpc.MergeTablesRequest
- 45, // 254: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.combo_aggregate:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest
- 69, // 255: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.flatten:type_name -> io.deephaven.proto.backplane.grpc.FlattenRequest
- 71, // 256: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.run_chart_downsample:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest
- 36, // 257: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.cross_join:type_name -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest
- 37, // 258: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.natural_join:type_name -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest
- 38, // 259: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.exact_join:type_name -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest
- 39, // 260: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.left_join:type_name -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest
- 40, // 261: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.as_of_join:type_name -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest
- 14, // 262: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.fetch_table:type_name -> io.deephaven.proto.backplane.grpc.FetchTableRequest
- 15, // 263: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.apply_preview_columns:type_name -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest
- 72, // 264: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.create_input_table:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest
- 26, // 265: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_by:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest
- 73, // 266: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.where_in:type_name -> io.deephaven.proto.backplane.grpc.WhereInRequest
- 46, // 267: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate_all:type_name -> io.deephaven.proto.backplane.grpc.AggregateAllRequest
- 48, // 268: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate:type_name -> io.deephaven.proto.backplane.grpc.AggregateRequest
- 34, // 269: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot:type_name -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest
- 35, // 270: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot_when:type_name -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest
- 70, // 271: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.meta_table:type_name -> io.deephaven.proto.backplane.grpc.MetaTableRequest
- 44, // 272: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.range_join:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest
- 41, // 273: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
- 41, // 274: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.raj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
- 74, // 275: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.column_statistics:type_name -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest
- 43, // 276: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.multi_join:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest
- 75, // 277: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.slice:type_name -> io.deephaven.proto.backplane.grpc.SliceRequest
- 140, // 278: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:input_type -> io.deephaven.proto.backplane.grpc.Ticket
- 14, // 279: io.deephaven.proto.backplane.grpc.TableService.FetchTable:input_type -> io.deephaven.proto.backplane.grpc.FetchTableRequest
- 15, // 280: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:input_type -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest
- 18, // 281: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:input_type -> io.deephaven.proto.backplane.grpc.EmptyTableRequest
- 19, // 282: io.deephaven.proto.backplane.grpc.TableService.TimeTable:input_type -> io.deephaven.proto.backplane.grpc.TimeTableRequest
- 28, // 283: io.deephaven.proto.backplane.grpc.TableService.DropColumns:input_type -> io.deephaven.proto.backplane.grpc.DropColumnsRequest
- 20, // 284: io.deephaven.proto.backplane.grpc.TableService.Update:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 285: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 286: io.deephaven.proto.backplane.grpc.TableService.View:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 287: io.deephaven.proto.backplane.grpc.TableService.UpdateView:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 20, // 288: io.deephaven.proto.backplane.grpc.TableService.Select:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
- 26, // 289: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:input_type -> io.deephaven.proto.backplane.grpc.UpdateByRequest
- 27, // 290: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:input_type -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest
- 52, // 291: io.deephaven.proto.backplane.grpc.TableService.Filter:input_type -> io.deephaven.proto.backplane.grpc.FilterTableRequest
- 29, // 292: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:input_type -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest
- 51, // 293: io.deephaven.proto.backplane.grpc.TableService.Sort:input_type -> io.deephaven.proto.backplane.grpc.SortTableRequest
- 30, // 294: io.deephaven.proto.backplane.grpc.TableService.Head:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
- 30, // 295: io.deephaven.proto.backplane.grpc.TableService.Tail:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
- 31, // 296: io.deephaven.proto.backplane.grpc.TableService.HeadBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
- 31, // 297: io.deephaven.proto.backplane.grpc.TableService.TailBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
- 32, // 298: io.deephaven.proto.backplane.grpc.TableService.Ungroup:input_type -> io.deephaven.proto.backplane.grpc.UngroupRequest
- 33, // 299: io.deephaven.proto.backplane.grpc.TableService.MergeTables:input_type -> io.deephaven.proto.backplane.grpc.MergeTablesRequest
- 36, // 300: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:input_type -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest
- 37, // 301: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:input_type -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest
- 38, // 302: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:input_type -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest
- 39, // 303: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:input_type -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest
- 40, // 304: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:input_type -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest
- 41, // 305: io.deephaven.proto.backplane.grpc.TableService.AjTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
- 41, // 306: io.deephaven.proto.backplane.grpc.TableService.RajTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
- 43, // 307: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:input_type -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest
- 44, // 308: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:input_type -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest
- 45, // 309: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:input_type -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest
- 46, // 310: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:input_type -> io.deephaven.proto.backplane.grpc.AggregateAllRequest
- 48, // 311: io.deephaven.proto.backplane.grpc.TableService.Aggregate:input_type -> io.deephaven.proto.backplane.grpc.AggregateRequest
- 34, // 312: io.deephaven.proto.backplane.grpc.TableService.Snapshot:input_type -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest
- 35, // 313: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:input_type -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest
- 69, // 314: io.deephaven.proto.backplane.grpc.TableService.Flatten:input_type -> io.deephaven.proto.backplane.grpc.FlattenRequest
- 71, // 315: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:input_type -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest
- 72, // 316: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:input_type -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest
- 73, // 317: io.deephaven.proto.backplane.grpc.TableService.WhereIn:input_type -> io.deephaven.proto.backplane.grpc.WhereInRequest
- 76, // 318: io.deephaven.proto.backplane.grpc.TableService.Batch:input_type -> io.deephaven.proto.backplane.grpc.BatchTableRequest
- 16, // 319: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:input_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest
- 53, // 320: io.deephaven.proto.backplane.grpc.TableService.SeekRow:input_type -> io.deephaven.proto.backplane.grpc.SeekRowRequest
- 70, // 321: io.deephaven.proto.backplane.grpc.TableService.MetaTable:input_type -> io.deephaven.proto.backplane.grpc.MetaTableRequest
- 74, // 322: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:input_type -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest
- 75, // 323: io.deephaven.proto.backplane.grpc.TableService.Slice:input_type -> io.deephaven.proto.backplane.grpc.SliceRequest
- 13, // 324: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 325: io.deephaven.proto.backplane.grpc.TableService.FetchTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 326: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 327: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 328: io.deephaven.proto.backplane.grpc.TableService.TimeTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 329: io.deephaven.proto.backplane.grpc.TableService.DropColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 330: io.deephaven.proto.backplane.grpc.TableService.Update:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 331: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 332: io.deephaven.proto.backplane.grpc.TableService.View:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 333: io.deephaven.proto.backplane.grpc.TableService.UpdateView:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 334: io.deephaven.proto.backplane.grpc.TableService.Select:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 335: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 336: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 337: io.deephaven.proto.backplane.grpc.TableService.Filter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 338: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 339: io.deephaven.proto.backplane.grpc.TableService.Sort:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 340: io.deephaven.proto.backplane.grpc.TableService.Head:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 341: io.deephaven.proto.backplane.grpc.TableService.Tail:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 342: io.deephaven.proto.backplane.grpc.TableService.HeadBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 343: io.deephaven.proto.backplane.grpc.TableService.TailBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 344: io.deephaven.proto.backplane.grpc.TableService.Ungroup:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 345: io.deephaven.proto.backplane.grpc.TableService.MergeTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 346: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 347: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 348: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 349: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 350: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 351: io.deephaven.proto.backplane.grpc.TableService.AjTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 352: io.deephaven.proto.backplane.grpc.TableService.RajTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 353: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 354: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 355: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 356: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 357: io.deephaven.proto.backplane.grpc.TableService.Aggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 358: io.deephaven.proto.backplane.grpc.TableService.Snapshot:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 359: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 360: io.deephaven.proto.backplane.grpc.TableService.Flatten:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 361: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 362: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 363: io.deephaven.proto.backplane.grpc.TableService.WhereIn:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 364: io.deephaven.proto.backplane.grpc.TableService.Batch:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 17, // 365: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage
- 54, // 366: io.deephaven.proto.backplane.grpc.TableService.SeekRow:output_type -> io.deephaven.proto.backplane.grpc.SeekRowResponse
- 13, // 367: io.deephaven.proto.backplane.grpc.TableService.MetaTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 368: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 13, // 369: io.deephaven.proto.backplane.grpc.TableService.Slice:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
- 324, // [324:370] is the sub-list for method output_type
- 278, // [278:324] is the sub-list for method input_type
- 278, // [278:278] is the sub-list for extension type_name
- 278, // [278:278] is the sub-list for extension extendee
- 0, // [0:278] is the sub-list for field type_name
+ 104, // 196: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.rolling_count_where:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountWhere
+ 105, // 197: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.count_where:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeCountWhere
+ 24, // 198: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
+ 23, // 199: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEma.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 24, // 200: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
+ 23, // 201: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEms.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 24, // 202: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
+ 23, // 203: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMin.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 24, // 204: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
+ 23, // 205: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMax.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 24, // 206: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByEmOptions
+ 23, // 207: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStd.window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 25, // 208: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDelta.options:type_name -> io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions
+ 23, // 209: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 210: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSum.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 211: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 212: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroup.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 213: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 214: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 215: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 216: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMin.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 217: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 218: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMax.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 219: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 220: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProduct.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 221: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 222: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCount.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 223: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 224: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStd.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 225: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 226: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvg.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 227: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 228: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 229: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountWhere.reverse_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 23, // 230: io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountWhere.forward_window_scale:type_name -> io.deephaven.proto.backplane.grpc.UpdateByWindowScale
+ 9, // 231: io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate.type:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType
+ 114, // 232: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSorted.columns:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn
+ 117, // 233: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUnique.non_unique_sentinel:type_name -> io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel
+ 2, // 234: io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel.null_value:type_name -> io.deephaven.proto.backplane.grpc.NullValue
+ 47, // 235: io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumns.spec:type_name -> io.deephaven.proto.backplane.grpc.AggSpec
+ 21, // 236: io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormula.selectable:type_name -> io.deephaven.proto.backplane.grpc.Selectable
+ 138, // 237: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_append_only:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnly
+ 139, // 238: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.in_memory_key_backed:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBacked
+ 140, // 239: io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.blink:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.Blink
+ 18, // 240: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.empty_table:type_name -> io.deephaven.proto.backplane.grpc.EmptyTableRequest
+ 19, // 241: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.time_table:type_name -> io.deephaven.proto.backplane.grpc.TimeTableRequest
+ 28, // 242: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.drop_columns:type_name -> io.deephaven.proto.backplane.grpc.DropColumnsRequest
+ 20, // 243: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 244: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.lazy_update:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 245: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 246: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_view:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 247: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select:type_name -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 27, // 248: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.select_distinct:type_name -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest
+ 52, // 249: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.filter:type_name -> io.deephaven.proto.backplane.grpc.FilterTableRequest
+ 29, // 250: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.unstructured_filter:type_name -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest
+ 51, // 251: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.sort:type_name -> io.deephaven.proto.backplane.grpc.SortTableRequest
+ 30, // 252: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
+ 30, // 253: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
+ 31, // 254: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.head_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
+ 31, // 255: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.tail_by:type_name -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
+ 32, // 256: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.ungroup:type_name -> io.deephaven.proto.backplane.grpc.UngroupRequest
+ 33, // 257: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.merge:type_name -> io.deephaven.proto.backplane.grpc.MergeTablesRequest
+ 45, // 258: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.combo_aggregate:type_name -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest
+ 69, // 259: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.flatten:type_name -> io.deephaven.proto.backplane.grpc.FlattenRequest
+ 71, // 260: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.run_chart_downsample:type_name -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest
+ 36, // 261: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.cross_join:type_name -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest
+ 37, // 262: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.natural_join:type_name -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest
+ 38, // 263: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.exact_join:type_name -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest
+ 39, // 264: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.left_join:type_name -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest
+ 40, // 265: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.as_of_join:type_name -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest
+ 14, // 266: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.fetch_table:type_name -> io.deephaven.proto.backplane.grpc.FetchTableRequest
+ 15, // 267: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.apply_preview_columns:type_name -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest
+ 72, // 268: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.create_input_table:type_name -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest
+ 26, // 269: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.update_by:type_name -> io.deephaven.proto.backplane.grpc.UpdateByRequest
+ 73, // 270: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.where_in:type_name -> io.deephaven.proto.backplane.grpc.WhereInRequest
+ 46, // 271: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate_all:type_name -> io.deephaven.proto.backplane.grpc.AggregateAllRequest
+ 48, // 272: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aggregate:type_name -> io.deephaven.proto.backplane.grpc.AggregateRequest
+ 34, // 273: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot:type_name -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest
+ 35, // 274: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.snapshot_when:type_name -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest
+ 70, // 275: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.meta_table:type_name -> io.deephaven.proto.backplane.grpc.MetaTableRequest
+ 44, // 276: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.range_join:type_name -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest
+ 41, // 277: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.aj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
+ 41, // 278: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.raj:type_name -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
+ 74, // 279: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.column_statistics:type_name -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest
+ 43, // 280: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.multi_join:type_name -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest
+ 75, // 281: io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation.slice:type_name -> io.deephaven.proto.backplane.grpc.SliceRequest
+ 142, // 282: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:input_type -> io.deephaven.proto.backplane.grpc.Ticket
+ 14, // 283: io.deephaven.proto.backplane.grpc.TableService.FetchTable:input_type -> io.deephaven.proto.backplane.grpc.FetchTableRequest
+ 15, // 284: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:input_type -> io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest
+ 18, // 285: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:input_type -> io.deephaven.proto.backplane.grpc.EmptyTableRequest
+ 19, // 286: io.deephaven.proto.backplane.grpc.TableService.TimeTable:input_type -> io.deephaven.proto.backplane.grpc.TimeTableRequest
+ 28, // 287: io.deephaven.proto.backplane.grpc.TableService.DropColumns:input_type -> io.deephaven.proto.backplane.grpc.DropColumnsRequest
+ 20, // 288: io.deephaven.proto.backplane.grpc.TableService.Update:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 289: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 290: io.deephaven.proto.backplane.grpc.TableService.View:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 291: io.deephaven.proto.backplane.grpc.TableService.UpdateView:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 20, // 292: io.deephaven.proto.backplane.grpc.TableService.Select:input_type -> io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest
+ 26, // 293: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:input_type -> io.deephaven.proto.backplane.grpc.UpdateByRequest
+ 27, // 294: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:input_type -> io.deephaven.proto.backplane.grpc.SelectDistinctRequest
+ 52, // 295: io.deephaven.proto.backplane.grpc.TableService.Filter:input_type -> io.deephaven.proto.backplane.grpc.FilterTableRequest
+ 29, // 296: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:input_type -> io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest
+ 51, // 297: io.deephaven.proto.backplane.grpc.TableService.Sort:input_type -> io.deephaven.proto.backplane.grpc.SortTableRequest
+ 30, // 298: io.deephaven.proto.backplane.grpc.TableService.Head:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
+ 30, // 299: io.deephaven.proto.backplane.grpc.TableService.Tail:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailRequest
+ 31, // 300: io.deephaven.proto.backplane.grpc.TableService.HeadBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
+ 31, // 301: io.deephaven.proto.backplane.grpc.TableService.TailBy:input_type -> io.deephaven.proto.backplane.grpc.HeadOrTailByRequest
+ 32, // 302: io.deephaven.proto.backplane.grpc.TableService.Ungroup:input_type -> io.deephaven.proto.backplane.grpc.UngroupRequest
+ 33, // 303: io.deephaven.proto.backplane.grpc.TableService.MergeTables:input_type -> io.deephaven.proto.backplane.grpc.MergeTablesRequest
+ 36, // 304: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:input_type -> io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest
+ 37, // 305: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:input_type -> io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest
+ 38, // 306: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:input_type -> io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest
+ 39, // 307: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:input_type -> io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest
+ 40, // 308: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:input_type -> io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest
+ 41, // 309: io.deephaven.proto.backplane.grpc.TableService.AjTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
+ 41, // 310: io.deephaven.proto.backplane.grpc.TableService.RajTables:input_type -> io.deephaven.proto.backplane.grpc.AjRajTablesRequest
+ 43, // 311: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:input_type -> io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest
+ 44, // 312: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:input_type -> io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest
+ 45, // 313: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:input_type -> io.deephaven.proto.backplane.grpc.ComboAggregateRequest
+ 46, // 314: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:input_type -> io.deephaven.proto.backplane.grpc.AggregateAllRequest
+ 48, // 315: io.deephaven.proto.backplane.grpc.TableService.Aggregate:input_type -> io.deephaven.proto.backplane.grpc.AggregateRequest
+ 34, // 316: io.deephaven.proto.backplane.grpc.TableService.Snapshot:input_type -> io.deephaven.proto.backplane.grpc.SnapshotTableRequest
+ 35, // 317: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:input_type -> io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest
+ 69, // 318: io.deephaven.proto.backplane.grpc.TableService.Flatten:input_type -> io.deephaven.proto.backplane.grpc.FlattenRequest
+ 71, // 319: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:input_type -> io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest
+ 72, // 320: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:input_type -> io.deephaven.proto.backplane.grpc.CreateInputTableRequest
+ 73, // 321: io.deephaven.proto.backplane.grpc.TableService.WhereIn:input_type -> io.deephaven.proto.backplane.grpc.WhereInRequest
+ 76, // 322: io.deephaven.proto.backplane.grpc.TableService.Batch:input_type -> io.deephaven.proto.backplane.grpc.BatchTableRequest
+ 16, // 323: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:input_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest
+ 53, // 324: io.deephaven.proto.backplane.grpc.TableService.SeekRow:input_type -> io.deephaven.proto.backplane.grpc.SeekRowRequest
+ 70, // 325: io.deephaven.proto.backplane.grpc.TableService.MetaTable:input_type -> io.deephaven.proto.backplane.grpc.MetaTableRequest
+ 74, // 326: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:input_type -> io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest
+ 75, // 327: io.deephaven.proto.backplane.grpc.TableService.Slice:input_type -> io.deephaven.proto.backplane.grpc.SliceRequest
+ 13, // 328: io.deephaven.proto.backplane.grpc.TableService.GetExportedTableCreationResponse:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 329: io.deephaven.proto.backplane.grpc.TableService.FetchTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 330: io.deephaven.proto.backplane.grpc.TableService.ApplyPreviewColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 331: io.deephaven.proto.backplane.grpc.TableService.EmptyTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 332: io.deephaven.proto.backplane.grpc.TableService.TimeTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 333: io.deephaven.proto.backplane.grpc.TableService.DropColumns:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 334: io.deephaven.proto.backplane.grpc.TableService.Update:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 335: io.deephaven.proto.backplane.grpc.TableService.LazyUpdate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 336: io.deephaven.proto.backplane.grpc.TableService.View:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 337: io.deephaven.proto.backplane.grpc.TableService.UpdateView:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 338: io.deephaven.proto.backplane.grpc.TableService.Select:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 339: io.deephaven.proto.backplane.grpc.TableService.UpdateBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 340: io.deephaven.proto.backplane.grpc.TableService.SelectDistinct:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 341: io.deephaven.proto.backplane.grpc.TableService.Filter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 342: io.deephaven.proto.backplane.grpc.TableService.UnstructuredFilter:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 343: io.deephaven.proto.backplane.grpc.TableService.Sort:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 344: io.deephaven.proto.backplane.grpc.TableService.Head:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 345: io.deephaven.proto.backplane.grpc.TableService.Tail:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 346: io.deephaven.proto.backplane.grpc.TableService.HeadBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 347: io.deephaven.proto.backplane.grpc.TableService.TailBy:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 348: io.deephaven.proto.backplane.grpc.TableService.Ungroup:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 349: io.deephaven.proto.backplane.grpc.TableService.MergeTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 350: io.deephaven.proto.backplane.grpc.TableService.CrossJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 351: io.deephaven.proto.backplane.grpc.TableService.NaturalJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 352: io.deephaven.proto.backplane.grpc.TableService.ExactJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 353: io.deephaven.proto.backplane.grpc.TableService.LeftJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 354: io.deephaven.proto.backplane.grpc.TableService.AsOfJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 355: io.deephaven.proto.backplane.grpc.TableService.AjTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 356: io.deephaven.proto.backplane.grpc.TableService.RajTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 357: io.deephaven.proto.backplane.grpc.TableService.MultiJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 358: io.deephaven.proto.backplane.grpc.TableService.RangeJoinTables:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 359: io.deephaven.proto.backplane.grpc.TableService.ComboAggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 360: io.deephaven.proto.backplane.grpc.TableService.AggregateAll:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 361: io.deephaven.proto.backplane.grpc.TableService.Aggregate:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 362: io.deephaven.proto.backplane.grpc.TableService.Snapshot:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 363: io.deephaven.proto.backplane.grpc.TableService.SnapshotWhen:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 364: io.deephaven.proto.backplane.grpc.TableService.Flatten:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 365: io.deephaven.proto.backplane.grpc.TableService.RunChartDownsample:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 366: io.deephaven.proto.backplane.grpc.TableService.CreateInputTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 367: io.deephaven.proto.backplane.grpc.TableService.WhereIn:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 368: io.deephaven.proto.backplane.grpc.TableService.Batch:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 17, // 369: io.deephaven.proto.backplane.grpc.TableService.ExportedTableUpdates:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage
+ 54, // 370: io.deephaven.proto.backplane.grpc.TableService.SeekRow:output_type -> io.deephaven.proto.backplane.grpc.SeekRowResponse
+ 13, // 371: io.deephaven.proto.backplane.grpc.TableService.MetaTable:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 372: io.deephaven.proto.backplane.grpc.TableService.ComputeColumnStatistics:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 13, // 373: io.deephaven.proto.backplane.grpc.TableService.Slice:output_type -> io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse
+ 328, // [328:374] is the sub-list for method output_type
+ 282, // [282:328] is the sub-list for method input_type
+ 282, // [282:282] is the sub-list for extension type_name
+ 282, // [282:282] is the sub-list for extension extendee
+ 0, // [0:282] is the sub-list for field type_name
}
func init() { file_deephaven_core_proto_table_proto_init() }
@@ -14382,7 +14598,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*ComboAggregateRequest_Aggregate); i {
+ switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByRollingCountWhere); i {
case 0:
return &v.state
case 1:
@@ -14394,7 +14610,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecApproximatePercentile); i {
+ switch v := v.(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_UpdateByCumulativeCountWhere); i {
case 0:
return &v.state
case 1:
@@ -14406,7 +14622,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecCountDistinct); i {
+ switch v := v.(*ComboAggregateRequest_Aggregate); i {
case 0:
return &v.state
case 1:
@@ -14418,7 +14634,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecDistinct); i {
+ switch v := v.(*AggSpec_AggSpecApproximatePercentile); i {
case 0:
return &v.state
case 1:
@@ -14430,7 +14646,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecFormula); i {
+ switch v := v.(*AggSpec_AggSpecCountDistinct); i {
case 0:
return &v.state
case 1:
@@ -14442,7 +14658,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecMedian); i {
+ switch v := v.(*AggSpec_AggSpecDistinct); i {
case 0:
return &v.state
case 1:
@@ -14454,7 +14670,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecPercentile); i {
+ switch v := v.(*AggSpec_AggSpecFormula); i {
case 0:
return &v.state
case 1:
@@ -14466,7 +14682,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecSorted); i {
+ switch v := v.(*AggSpec_AggSpecMedian); i {
case 0:
return &v.state
case 1:
@@ -14478,7 +14694,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[100].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecSortedColumn); i {
+ switch v := v.(*AggSpec_AggSpecPercentile); i {
case 0:
return &v.state
case 1:
@@ -14490,7 +14706,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[101].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecTDigest); i {
+ switch v := v.(*AggSpec_AggSpecSorted); i {
case 0:
return &v.state
case 1:
@@ -14502,7 +14718,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[102].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecUnique); i {
+ switch v := v.(*AggSpec_AggSpecSortedColumn); i {
case 0:
return &v.state
case 1:
@@ -14514,7 +14730,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[103].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecNonUniqueSentinel); i {
+ switch v := v.(*AggSpec_AggSpecTDigest); i {
case 0:
return &v.state
case 1:
@@ -14526,7 +14742,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[104].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecWeighted); i {
+ switch v := v.(*AggSpec_AggSpecUnique); i {
case 0:
return &v.state
case 1:
@@ -14538,7 +14754,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[105].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecAbsSum); i {
+ switch v := v.(*AggSpec_AggSpecNonUniqueSentinel); i {
case 0:
return &v.state
case 1:
@@ -14550,7 +14766,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[106].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecAvg); i {
+ switch v := v.(*AggSpec_AggSpecWeighted); i {
case 0:
return &v.state
case 1:
@@ -14562,7 +14778,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[107].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecFirst); i {
+ switch v := v.(*AggSpec_AggSpecAbsSum); i {
case 0:
return &v.state
case 1:
@@ -14574,7 +14790,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[108].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecFreeze); i {
+ switch v := v.(*AggSpec_AggSpecAvg); i {
case 0:
return &v.state
case 1:
@@ -14586,7 +14802,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecGroup); i {
+ switch v := v.(*AggSpec_AggSpecFirst); i {
case 0:
return &v.state
case 1:
@@ -14598,7 +14814,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecLast); i {
+ switch v := v.(*AggSpec_AggSpecFreeze); i {
case 0:
return &v.state
case 1:
@@ -14610,7 +14826,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecMax); i {
+ switch v := v.(*AggSpec_AggSpecGroup); i {
case 0:
return &v.state
case 1:
@@ -14622,7 +14838,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecMin); i {
+ switch v := v.(*AggSpec_AggSpecLast); i {
case 0:
return &v.state
case 1:
@@ -14634,7 +14850,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[113].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecStd); i {
+ switch v := v.(*AggSpec_AggSpecMax); i {
case 0:
return &v.state
case 1:
@@ -14646,7 +14862,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[114].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecSum); i {
+ switch v := v.(*AggSpec_AggSpecMin); i {
case 0:
return &v.state
case 1:
@@ -14658,7 +14874,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[115].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*AggSpec_AggSpecVar); i {
+ switch v := v.(*AggSpec_AggSpecStd); i {
case 0:
return &v.state
case 1:
@@ -14670,7 +14886,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[116].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*Aggregation_AggregationColumns); i {
+ switch v := v.(*AggSpec_AggSpecSum); i {
case 0:
return &v.state
case 1:
@@ -14682,7 +14898,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[117].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*Aggregation_AggregationCount); i {
+ switch v := v.(*AggSpec_AggSpecVar); i {
case 0:
return &v.state
case 1:
@@ -14694,7 +14910,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[118].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*Aggregation_AggregationCountWhere); i {
+ switch v := v.(*Aggregation_AggregationColumns); i {
case 0:
return &v.state
case 1:
@@ -14706,7 +14922,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[119].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*Aggregation_AggregationRowKey); i {
+ switch v := v.(*Aggregation_AggregationCount); i {
case 0:
return &v.state
case 1:
@@ -14718,7 +14934,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[120].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*Aggregation_AggregationPartition); i {
+ switch v := v.(*Aggregation_AggregationCountWhere); i {
case 0:
return &v.state
case 1:
@@ -14730,7 +14946,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[121].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*Aggregation_AggregationFormula); i {
+ switch v := v.(*Aggregation_AggregationRowKey); i {
case 0:
return &v.state
case 1:
@@ -14742,7 +14958,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[122].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*RunChartDownsampleRequest_ZoomRange); i {
+ switch v := v.(*Aggregation_AggregationPartition); i {
case 0:
return &v.state
case 1:
@@ -14754,7 +14970,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[123].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*CreateInputTableRequest_InputTableKind); i {
+ switch v := v.(*Aggregation_AggregationFormula); i {
case 0:
return &v.state
case 1:
@@ -14766,7 +14982,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[124].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly); i {
+ switch v := v.(*RunChartDownsampleRequest_ZoomRange); i {
case 0:
return &v.state
case 1:
@@ -14778,7 +14994,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[125].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked); i {
+ switch v := v.(*CreateInputTableRequest_InputTableKind); i {
case 0:
return &v.state
case 1:
@@ -14790,7 +15006,7 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[126].Exporter = func(v interface{}, i int) interface{} {
- switch v := v.(*CreateInputTableRequest_InputTableKind_Blink); i {
+ switch v := v.(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly); i {
case 0:
return &v.state
case 1:
@@ -14802,6 +15018,30 @@ func file_deephaven_core_proto_table_proto_init() {
}
}
file_deephaven_core_proto_table_proto_msgTypes[127].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_core_proto_table_proto_msgTypes[128].Exporter = func(v interface{}, i int) interface{} {
+ switch v := v.(*CreateInputTableRequest_InputTableKind_Blink); i {
+ case 0:
+ return &v.state
+ case 1:
+ return &v.sizeCache
+ case 2:
+ return &v.unknownFields
+ default:
+ return nil
+ }
+ }
+ file_deephaven_core_proto_table_proto_msgTypes[129].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BatchTableRequest_Operation); i {
case 0:
return &v.state
@@ -14923,10 +15163,12 @@ func file_deephaven_core_proto_table_proto_init() {
(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingStd)(nil),
(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingWavg)(nil),
(*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingFormula)(nil),
+ (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_RollingCountWhere)(nil),
+ (*UpdateByRequest_UpdateByOperation_UpdateByColumn_UpdateBySpec_CountWhere)(nil),
}
- file_deephaven_core_proto_table_proto_msgTypes[93].OneofWrappers = []interface{}{}
- file_deephaven_core_proto_table_proto_msgTypes[101].OneofWrappers = []interface{}{}
- file_deephaven_core_proto_table_proto_msgTypes[103].OneofWrappers = []interface{}{
+ file_deephaven_core_proto_table_proto_msgTypes[95].OneofWrappers = []interface{}{}
+ file_deephaven_core_proto_table_proto_msgTypes[103].OneofWrappers = []interface{}{}
+ file_deephaven_core_proto_table_proto_msgTypes[105].OneofWrappers = []interface{}{
(*AggSpec_AggSpecNonUniqueSentinel_NullValue)(nil),
(*AggSpec_AggSpecNonUniqueSentinel_StringValue)(nil),
(*AggSpec_AggSpecNonUniqueSentinel_IntValue)(nil),
@@ -14938,13 +15180,13 @@ func file_deephaven_core_proto_table_proto_init() {
(*AggSpec_AggSpecNonUniqueSentinel_ShortValue)(nil),
(*AggSpec_AggSpecNonUniqueSentinel_CharValue)(nil),
}
- file_deephaven_core_proto_table_proto_msgTypes[122].OneofWrappers = []interface{}{}
- file_deephaven_core_proto_table_proto_msgTypes[123].OneofWrappers = []interface{}{
+ file_deephaven_core_proto_table_proto_msgTypes[124].OneofWrappers = []interface{}{}
+ file_deephaven_core_proto_table_proto_msgTypes[125].OneofWrappers = []interface{}{
(*CreateInputTableRequest_InputTableKind_InMemoryAppendOnly_)(nil),
(*CreateInputTableRequest_InputTableKind_InMemoryKeyBacked_)(nil),
(*CreateInputTableRequest_InputTableKind_Blink_)(nil),
}
- file_deephaven_core_proto_table_proto_msgTypes[127].OneofWrappers = []interface{}{
+ file_deephaven_core_proto_table_proto_msgTypes[129].OneofWrappers = []interface{}{
(*BatchTableRequest_Operation_EmptyTable)(nil),
(*BatchTableRequest_Operation_TimeTable)(nil),
(*BatchTableRequest_Operation_DropColumns)(nil),
@@ -14994,7 +15236,7 @@ func file_deephaven_core_proto_table_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_deephaven_core_proto_table_proto_rawDesc,
NumEnums: 12,
- NumMessages: 128,
+ NumMessages: 130,
NumExtensions: 0,
NumServices: 1,
},
diff --git a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java
index b0ddd59a18a..179c8104a56 100644
--- a/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java
+++ b/java-client/session/src/main/java/io/deephaven/client/impl/UpdateByBuilder.java
@@ -6,6 +6,7 @@
import io.deephaven.api.ColumnName;
import io.deephaven.api.Strings;
import io.deephaven.api.Pair;
+import io.deephaven.api.filter.Filter;
import io.deephaven.api.updateby.*;
import io.deephaven.api.updateby.BadDataBehavior;
import io.deephaven.api.updateby.spec.*;
@@ -17,6 +18,8 @@
import java.math.MathContext;
import java.math.RoundingMode;
+import java.util.Collection;
+import java.util.stream.Collectors;
class UpdateByBuilder {
@@ -188,6 +191,19 @@ public UpdateByColumn.UpdateBySpec visit(CumProdSpec spec) {
.build();
}
+ @Override
+ public UpdateByColumn.UpdateBySpec visit(CumCountWhereSpec spec) {
+ final Collection filters = Filter.extractAnds(spec.filter()).stream()
+ .map(Strings::of)
+ .collect(Collectors.toList());
+ return UpdateByColumn.UpdateBySpec.newBuilder()
+ .setCountWhere(UpdateByCumulativeCountWhere.newBuilder()
+ .setResultColumn(spec.column().name())
+ .addAllFilters(filters)
+ .build())
+ .build();
+ }
+
@Override
public UpdateByColumn.UpdateBySpec visit(EmStdSpec spec) {
UpdateByEmStd.Builder builder = UpdateByEmStd.newBuilder().setWindowScale(adapt(spec.windowScale()));
@@ -282,6 +298,22 @@ public UpdateByColumn.UpdateBySpec visit(RollingCountSpec rs) {
.build();
}
+ @Override
+ public UpdateByColumn.UpdateBySpec visit(RollingCountWhereSpec rs) {
+ final Collection filters = Filter.extractAnds(rs.filter()).stream()
+ .map(Strings::of)
+ .collect(Collectors.toList());
+ final UpdateByRollingCountWhere.Builder builder =
+ UpdateByRollingCountWhere.newBuilder()
+ .setReverseWindowScale(adapt(rs.revWindowScale()))
+ .setForwardWindowScale(adapt(rs.fwdWindowScale()))
+ .setResultColumn(rs.column().name())
+ .addAllFilters(filters);
+ return UpdateByColumn.UpdateBySpec.newBuilder()
+ .setRollingCountWhere(builder.build())
+ .build();
+ }
+
@Override
public UpdateByColumn.UpdateBySpec visit(RollingStdSpec rs) {
final UpdateByRollingStd.Builder builder =
diff --git a/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java b/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java
index 1259378abd4..bb758e72c71 100644
--- a/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java
+++ b/java-client/session/src/test/java/io/deephaven/client/impl/UpdateBySpecBuilderTest.java
@@ -123,6 +123,15 @@ public UpdateByColumn.UpdateBySpec visit(CumProdSpec spec) {
.build();
}
+ @Override
+ public UpdateByColumn.UpdateBySpec visit(CumCountWhereSpec spec) {
+ return UpdateByColumn.UpdateBySpec.newBuilder()
+ .setCountWhere(UpdateByCumulativeCountWhere.newBuilder()
+ .setResultColumn("count")
+ .addFilters("x > 5"))
+ .build();
+ }
+
@Override
public UpdateByColumn.UpdateBySpec visit(DeltaSpec spec) {
if (spec.deltaControl().isPresent()) {
@@ -321,6 +330,25 @@ public UpdateByColumn.UpdateBySpec visit(RollingFormulaSpec spec) {
.build())
.build();
}
+
+ @Override
+ public UpdateByColumn.UpdateBySpec visit(RollingCountWhereSpec spec) {
+ return UpdateByColumn.UpdateBySpec
+ .newBuilder().setRollingCountWhere(
+ UpdateByColumn.UpdateBySpec.UpdateByRollingCountWhere.newBuilder()
+ .setReverseWindowScale(UpdateByWindowScale.newBuilder()
+ .setTime(UpdateByWindowScale.UpdateByWindowTime.newBuilder()
+ .setColumn("Timestamp").setNanos(1).build())
+ .build())
+ .setForwardWindowScale(UpdateByWindowScale.newBuilder()
+ .setTime(UpdateByWindowScale.UpdateByWindowTime.newBuilder()
+ .setColumn("Timestamp").setNanos(1).build())
+ .build())
+ .setResultColumn("count")
+ .addFilters("x > 5")
+ .build())
+ .build();
+ }
}
@Test
@@ -428,6 +456,11 @@ void cumulativeProd() {
check(CumProdSpec.of());
}
+ @Test
+ void cumulativeCountWhere() {
+ check(CumCountWhereSpec.of("count", "x > 5"));
+ }
+
@Test
void fillBy() {
check(FillBySpec.of());
@@ -658,6 +691,29 @@ void rollingFormula() {
.build());
}
+ @Test
+ void rollingCountWhere() {
+ check(RollingFormulaSpec.ofTime("Timestamp", Duration.ofNanos(1), Duration.ofNanos(2), "sum(x)", "x"),
+ UpdateByColumn.UpdateBySpec.newBuilder().setRollingFormula(
+ UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.newBuilder()
+ .setReverseWindowScale(time("Timestamp", 1))
+ .setForwardWindowScale(time("Timestamp", 2))
+ .setFormula("sum(x)")
+ .setParamToken("x")
+ .build())
+ .build());
+
+ check(RollingFormulaSpec.ofTicks(42L, 43L, "sum(x)", "x"),
+ UpdateByColumn.UpdateBySpec.newBuilder().setRollingFormula(
+ UpdateByColumn.UpdateBySpec.UpdateByRollingFormula.newBuilder()
+ .setReverseWindowScale(ticks(42L))
+ .setForwardWindowScale(ticks(43L))
+ .setFormula("sum(x)")
+ .setParamToken("x")
+ .build())
+ .build());
+ }
+
private static void check(UpdateBySpec spec) {
check(spec, spec.walk(ExpectedSpecVisitor.INSTANCE));
}
diff --git a/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto b/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto
index 7e89adf84bd..587e261b6c5 100644
--- a/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto
+++ b/proto/proto-backplane-grpc/src/main/proto/deephaven_core/proto/table.proto
@@ -593,6 +593,24 @@ message UpdateByRequest {
string param_token = 4;
}
+ message UpdateByRollingCountWhere {
+ // The reverse window for the rolling operation, may be in ticks or time-based.
+ UpdateByWindowScale reverse_window_scale = 1;
+ // The forward window for the rolling operation, may be in ticks or time-based.
+ UpdateByWindowScale forward_window_scale = 2;
+ // The output column name
+ string result_column = 3;
+ // The filters tp apply before counting
+ repeated string filters = 4;
+ }
+
+ message UpdateByCumulativeCountWhere {
+ // The output column name
+ string result_column = 1;
+ // The filters tp apply before counting
+ repeated string filters = 2;
+ }
+
oneof type {
UpdateByCumulativeSum sum = 1;
UpdateByCumulativeMin min = 2;
@@ -615,6 +633,8 @@ message UpdateByRequest {
UpdateByRollingStd rolling_std = 19;
UpdateByRollingWAvg rolling_wavg = 20;
UpdateByRollingFormula rolling_formula = 21;
+ UpdateByRollingCountWhere rolling_count_where = 22;
+ UpdateByCumulativeCountWhere count_where = 23;
}
}
diff --git a/py/client/deephaven_core/proto/table_pb2.py b/py/client/deephaven_core/proto/table_pb2.py
index 1442f9932a8..89284db0e02 100644
--- a/py/client/deephaven_core/proto/table_pb2.py
+++ b/py/client/deephaven_core/proto/table_pb2.py
@@ -15,7 +15,7 @@
from deephaven_core.proto import ticket_pb2 as deephaven__core_dot_proto_dot_ticket__pb2
-DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n deephaven_core/proto/table.proto\x12!io.deephaven.proto.backplane.grpc\x1a!deephaven_core/proto/ticket.proto\"l\n\x0eTableReference\x12;\n\x06ticket\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.TicketH\x00\x12\x16\n\x0c\x62\x61tch_offset\x18\x02 \x01(\x11H\x00\x42\x05\n\x03ref\"\xc6\x01\n\x1d\x45xportedTableCreationResponse\x12\x44\n\tresult_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07success\x18\x02 \x01(\x08\x12\x12\n\nerror_info\x18\x03 \x01(\t\x12\x15\n\rschema_header\x18\x04 \x01(\x0c\x12\x11\n\tis_static\x18\x05 \x01(\x08\x12\x10\n\x04size\x18\x06 \x01(\x12\x42\x02\x30\x01\"\x97\x01\n\x11\x46\x65tchTableRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\xa0\x01\n\x1a\x41pplyPreviewColumnsRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\x1d\n\x1b\x45xportedTableUpdatesRequest\"\x8c\x01\n\x1a\x45xportedTableUpdateMessage\x12<\n\texport_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x1e\n\x16update_failure_message\x18\x03 \x01(\t\"c\n\x11\x45mptyTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\"\xef\x01\n\x10TimeTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x1e\n\x10start_time_nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1b\n\x11start_time_string\x18\x05 \x01(\tH\x00\x12\x1a\n\x0cperiod_nanos\x18\x03 \x01(\x12\x42\x02\x30\x01H\x01\x12\x17\n\rperiod_string\x18\x06 \x01(\tH\x01\x12\x13\n\x0b\x62link_table\x18\x04 \x01(\x08\x42\x0c\n\nstart_timeB\x08\n\x06period\"\xb1\x01\n\x15SelectOrUpdateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_specs\x18\x03 \x03(\t\"#\n\nSelectable\x12\r\n\x03raw\x18\x01 \x01(\tH\x00\x42\x06\n\x04type\"\x8c\x02\n\x0bMathContext\x12\x11\n\tprecision\x18\x01 \x01(\x11\x12R\n\rrounding_mode\x18\x02 \x01(\x0e\x32;.io.deephaven.proto.backplane.grpc.MathContext.RoundingMode\"\x95\x01\n\x0cRoundingMode\x12\x1f\n\x1bROUNDING_MODE_NOT_SPECIFIED\x10\x00\x12\x06\n\x02UP\x10\x01\x12\x08\n\x04\x44OWN\x10\x02\x12\x0b\n\x07\x43\x45ILING\x10\x03\x12\t\n\x05\x46LOOR\x10\x04\x12\x0b\n\x07HALF_UP\x10\x05\x12\r\n\tHALF_DOWN\x10\x06\x12\r\n\tHALF_EVEN\x10\x07\x12\x0f\n\x0bUNNECESSARY\x10\x08\"\xdb\x02\n\x13UpdateByWindowScale\x12[\n\x05ticks\x18\x01 \x01(\x0b\x32J.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicksH\x00\x12Y\n\x04time\x18\x02 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTimeH\x00\x1a$\n\x13UpdateByWindowTicks\x12\r\n\x05ticks\x18\x01 \x01(\x01\x1a^\n\x12UpdateByWindowTime\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x13\n\x05nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x19\n\x0f\x64uration_string\x18\x03 \x01(\tH\x00\x42\x08\n\x06windowB\x06\n\x04type\"\xe1\x03\n\x11UpdateByEmOptions\x12I\n\ron_null_value\x18\x01 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_nan_value\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_null_time\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12R\n\x16on_negative_delta_time\x18\x04 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12N\n\x12on_zero_delta_time\x18\x05 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12I\n\x11\x62ig_value_context\x18\x06 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContext\"f\n\x14UpdateByDeltaOptions\x12N\n\rnull_behavior\x18\x01 \x01(\x0e\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByNullBehavior\"\x9b\x37\n\x0fUpdateByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12S\n\x07options\x18\x03 \x01(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOptions\x12X\n\noperations\x18\x04 \x03(\x0b\x32\x44.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation\x12\x18\n\x10group_by_columns\x18\x05 \x03(\t\x1a\xc3\x03\n\x0fUpdateByOptions\x12\x1c\n\x0fuse_redirection\x18\x01 \x01(\x08H\x00\x88\x01\x01\x12\x1b\n\x0e\x63hunk_capacity\x18\x02 \x01(\x05H\x01\x88\x01\x01\x12.\n!max_static_sparse_memory_overhead\x18\x03 \x01(\x01H\x02\x88\x01\x01\x12$\n\x17initial_hash_table_size\x18\x04 \x01(\x05H\x03\x88\x01\x01\x12 \n\x13maximum_load_factor\x18\x05 \x01(\x01H\x04\x88\x01\x01\x12\x1f\n\x12target_load_factor\x18\x06 \x01(\x01H\x05\x88\x01\x01\x12\x44\n\x0cmath_context\x18\x07 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContextB\x12\n\x10_use_redirectionB\x11\n\x0f_chunk_capacityB$\n\"_max_static_sparse_memory_overheadB\x1a\n\x18_initial_hash_table_sizeB\x16\n\x14_maximum_load_factorB\x15\n\x13_target_load_factor\x1a\xf4\x30\n\x11UpdateByOperation\x12\x65\n\x06\x63olumn\x18\x01 \x01(\x0b\x32S.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumnH\x00\x1a\xef/\n\x0eUpdateByColumn\x12n\n\x04spec\x18\x01 \x01(\x0b\x32`.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\xd7.\n\x0cUpdateBySpec\x12\x85\x01\n\x03sum\x18\x01 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeSumH\x00\x12\x85\x01\n\x03min\x18\x02 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMinH\x00\x12\x85\x01\n\x03max\x18\x03 \x01(\x0b\x32v.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeMaxH\x00\x12\x8d\x01\n\x07product\x18\x04 \x01(\x0b\x32z.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeProductH\x00\x12}\n\x04\x66ill\x18\x05 \x01(\x0b\x32m.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByFillH\x00\x12{\n\x03\x65ma\x18\x06 \x01(\x0b\x32l.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmaH\x00\x12\x8a\x01\n\x0brolling_sum\x18\x07 \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingSumH\x00\x12\x8e\x01\n\rrolling_group\x18\x08 \x01(\x0b\x32u.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingGroupH\x00\x12\x8a\x01\n\x0brolling_avg\x18\t \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingAvgH\x00\x12\x8a\x01\n\x0brolling_min\x18\n \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMinH\x00\x12\x8a\x01\n\x0brolling_max\x18\x0b \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingMaxH\x00\x12\x92\x01\n\x0frolling_product\x18\x0c \x01(\x0b\x32w.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingProductH\x00\x12\x7f\n\x05\x64\x65lta\x18\r \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByDeltaH\x00\x12{\n\x03\x65ms\x18\x0e \x01(\x0b\x32l.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmsH\x00\x12\x80\x01\n\x06\x65m_min\x18\x0f \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMinH\x00\x12\x80\x01\n\x06\x65m_max\x18\x10 \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmMaxH\x00\x12\x80\x01\n\x06\x65m_std\x18\x11 \x01(\x0b\x32n.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByEmStdH\x00\x12\x8e\x01\n\rrolling_count\x18\x12 \x01(\x0b\x32u.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountH\x00\x12\x8a\x01\n\x0brolling_std\x18\x13 \x01(\x0b\x32s.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingStdH\x00\x12\x8c\x01\n\x0crolling_wavg\x18\x14 \x01(\x0b\x32t.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingWAvgH\x00\x12\x92\x01\n\x0frolling_formula\x18\x15 \x01(\x0b\x32w.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingFormulaH\x00\x1a\x17\n\x15UpdateByCumulativeSum\x1a\x17\n\x15UpdateByCumulativeMin\x1a\x17\n\x15UpdateByCumulativeMax\x1a\x1b\n\x19UpdateByCumulativeProduct\x1a\x0e\n\x0cUpdateByFill\x1a\xa2\x01\n\x0bUpdateByEma\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa2\x01\n\x0bUpdateByEms\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmMin\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmMax\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xa4\x01\n\rUpdateByEmStd\x12\x45\n\x07options\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.UpdateByEmOptions\x12L\n\x0cwindow_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1aY\n\rUpdateByDelta\x12H\n\x07options\x18\x01 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByDeltaOptions\x1a\xc0\x01\n\x12UpdateByRollingSum\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc2\x01\n\x14UpdateByRollingGroup\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingAvg\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingMin\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingMax\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc4\x01\n\x16UpdateByRollingProduct\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc2\x01\n\x14UpdateByRollingCount\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xc0\x01\n\x12UpdateByRollingStd\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x1a\xd8\x01\n\x13UpdateByRollingWAvg\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12\x15\n\rweight_column\x18\x03 \x01(\t\x1a\xea\x01\n\x16UpdateByRollingFormula\x12T\n\x14reverse_window_scale\x18\x01 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12T\n\x14\x66orward_window_scale\x18\x02 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.UpdateByWindowScale\x12\x0f\n\x07\x66ormula\x18\x03 \x01(\t\x12\x13\n\x0bparam_token\x18\x04 \x01(\tB\x06\n\x04typeB\x06\n\x04type\"\xb1\x01\n\x15SelectDistinctRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xae\x01\n\x12\x44ropColumnsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xb5\x01\n\x1eUnstructuredFilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07\x66ilters\x18\x03 \x03(\t\"\xad\x01\n\x11HeadOrTailRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\"\xce\x01\n\x13HeadOrTailByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\x12\x1d\n\x15group_by_column_specs\x18\x04 \x03(\t\"\xc3\x01\n\x0eUngroupRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x11\n\tnull_fill\x18\x03 \x01(\x08\x12\x1a\n\x12\x63olumns_to_ungroup\x18\x04 \x03(\t\"\xad\x01\n\x12MergeTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x45\n\nsource_ids\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x12\n\nkey_column\x18\x03 \x01(\t\"\x9a\x01\n\x14SnapshotTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb1\x02\n\x18SnapshotWhenTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07\x62\x61se_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x45\n\ntrigger_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07initial\x18\x04 \x01(\x08\x12\x13\n\x0bincremental\x18\x05 \x01(\x08\x12\x0f\n\x07history\x18\x06 \x01(\x08\x12\x15\n\rstamp_columns\x18\x07 \x03(\t\"\xa7\x02\n\x16\x43rossJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\x14\n\x0creserve_bits\x18\x06 \x01(\x05\"\x93\x02\n\x18NaturalJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x91\x02\n\x16\x45xactJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x90\x02\n\x15LeftJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\xd1\x03\n\x15\x41sOfJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\\\n\x10\x61s_of_match_rule\x18\x07 \x01(\x0e\x32\x42.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.MatchRule\"]\n\tMatchRule\x12\x13\n\x0fLESS_THAN_EQUAL\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12GREATER_THAN_EQUAL\x10\x02\x12\x10\n\x0cGREATER_THAN\x10\x03\x1a\x02\x18\x01:\x02\x18\x01\"\xa6\x02\n\x12\x41jRajTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x14\n\x0c\x61s_of_column\x18\x05 \x01(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x06 \x03(\t\"\x88\x01\n\x0eMultiJoinInput\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x02 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x03 \x03(\t\"\xa4\x01\n\x16MultiJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x11multi_join_inputs\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.MultiJoinInput\"\xe0\x06\n\x16RangeJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x19\n\x11left_start_column\x18\x05 \x01(\t\x12\x62\n\x10range_start_rule\x18\x06 \x01(\x0e\x32H.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeStartRule\x12\x1a\n\x12right_range_column\x18\x07 \x01(\t\x12^\n\x0erange_end_rule\x18\x08 \x01(\x0e\x32\x46.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeEndRule\x12\x17\n\x0fleft_end_column\x18\t \x01(\t\x12\x44\n\x0c\x61ggregations\x18\n \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x13\n\x0brange_match\x18\x0b \x01(\t\"v\n\x0eRangeStartRule\x12\x15\n\x11START_UNSPECIFIED\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x02\x12&\n\"LESS_THAN_OR_EQUAL_ALLOW_PRECEDING\x10\x03\"{\n\x0cRangeEndRule\x12\x13\n\x0f\x45ND_UNSPECIFIED\x10\x00\x12\x10\n\x0cGREATER_THAN\x10\x01\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x02\x12)\n%GREATER_THAN_OR_EQUAL_ALLOW_FOLLOWING\x10\x03\"\xfe\x04\n\x15\x43omboAggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12V\n\naggregates\x18\x03 \x03(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\x12\x13\n\x0b\x66orce_combo\x18\x05 \x01(\x08\x1a\xad\x01\n\tAggregate\x12N\n\x04type\x18\x01 \x01(\x0e\x32@.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x12\n\npercentile\x18\x04 \x01(\x01\x12\x12\n\navg_median\x18\x05 \x01(\x08\"\xa5\x01\n\x07\x41ggType\x12\x07\n\x03SUM\x10\x00\x12\x0b\n\x07\x41\x42S_SUM\x10\x01\x12\t\n\x05GROUP\x10\x02\x12\x07\n\x03\x41VG\x10\x03\x12\t\n\x05\x43OUNT\x10\x04\x12\t\n\x05\x46IRST\x10\x05\x12\x08\n\x04LAST\x10\x06\x12\x07\n\x03MIN\x10\x07\x12\x07\n\x03MAX\x10\x08\x12\n\n\x06MEDIAN\x10\t\x12\x0e\n\nPERCENTILE\x10\n\x12\x07\n\x03STD\x10\x0b\x12\x07\n\x03VAR\x10\x0c\x12\x10\n\x0cWEIGHTED_AVG\x10\r:\x02\x18\x01\"\xed\x01\n\x13\x41ggregateAllRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x38\n\x04spec\x18\x03 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\"\xd7\x17\n\x07\x41ggSpec\x12K\n\x07\x61\x62s_sum\x18\x01 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSumH\x00\x12i\n\x16\x61pproximate_percentile\x18\x02 \x01(\x0b\x32G.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentileH\x00\x12\x44\n\x03\x61vg\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvgH\x00\x12Y\n\x0e\x63ount_distinct\x18\x04 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinctH\x00\x12N\n\x08\x64istinct\x18\x05 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinctH\x00\x12H\n\x05\x66irst\x18\x06 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirstH\x00\x12L\n\x07\x66ormula\x18\x07 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormulaH\x00\x12J\n\x06\x66reeze\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreezeH\x00\x12H\n\x05group\x18\t \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroupH\x00\x12\x46\n\x04last\x18\n \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLastH\x00\x12\x44\n\x03max\x18\x0b \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMaxH\x00\x12J\n\x06median\x18\x0c \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedianH\x00\x12\x44\n\x03min\x18\r \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMinH\x00\x12R\n\npercentile\x18\x0e \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentileH\x00\x12P\n\x0csorted_first\x18\x0f \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12O\n\x0bsorted_last\x18\x10 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12\x44\n\x03std\x18\x11 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStdH\x00\x12\x44\n\x03sum\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSumH\x00\x12M\n\x08t_digest\x18\x13 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigestH\x00\x12J\n\x06unique\x18\x14 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUniqueH\x00\x12R\n\x0cweighted_avg\x18\x15 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12R\n\x0cweighted_sum\x18\x16 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12\x44\n\x03var\x18\x17 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVarH\x00\x1a\\\n\x1c\x41ggSpecApproximatePercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x18\n\x0b\x63ompression\x18\x02 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a+\n\x14\x41ggSpecCountDistinct\x12\x13\n\x0b\x63ount_nulls\x18\x01 \x01(\x08\x1a(\n\x0f\x41ggSpecDistinct\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x1a\x36\n\x0e\x41ggSpecFormula\x12\x0f\n\x07\x66ormula\x18\x01 \x01(\t\x12\x13\n\x0bparam_token\x18\x02 \x01(\t\x1a/\n\rAggSpecMedian\x12\x1e\n\x16\x61verage_evenly_divided\x18\x01 \x01(\x08\x1aG\n\x11\x41ggSpecPercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x1e\n\x16\x61verage_evenly_divided\x18\x02 \x01(\x08\x1a`\n\rAggSpecSorted\x12O\n\x07\x63olumns\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn\x1a*\n\x13\x41ggSpecSortedColumn\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a:\n\x0e\x41ggSpecTDigest\x12\x18\n\x0b\x63ompression\x18\x01 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a\x88\x01\n\rAggSpecUnique\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x12`\n\x13non_unique_sentinel\x18\x02 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel\x1a\xb5\x02\n\x18\x41ggSpecNonUniqueSentinel\x12\x42\n\nnull_value\x18\x01 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.NullValueH\x00\x12\x16\n\x0cstring_value\x18\x02 \x01(\tH\x00\x12\x13\n\tint_value\x18\x03 \x01(\x11H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x15\n\x0b\x66loat_value\x18\x05 \x01(\x02H\x00\x12\x16\n\x0c\x64ouble_value\x18\x06 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x07 \x01(\x08H\x00\x12\x14\n\nbyte_value\x18\x08 \x01(\x11H\x00\x12\x15\n\x0bshort_value\x18\t \x01(\x11H\x00\x12\x14\n\nchar_value\x18\n \x01(\x11H\x00\x42\x06\n\x04type\x1a(\n\x0f\x41ggSpecWeighted\x12\x15\n\rweight_column\x18\x01 \x01(\t\x1a\x0f\n\rAggSpecAbsSum\x1a\x0c\n\nAggSpecAvg\x1a\x0e\n\x0c\x41ggSpecFirst\x1a\x0f\n\rAggSpecFreeze\x1a\x0e\n\x0c\x41ggSpecGroup\x1a\r\n\x0b\x41ggSpecLast\x1a\x0c\n\nAggSpecMax\x1a\x0c\n\nAggSpecMin\x1a\x0c\n\nAggSpecStd\x1a\x0c\n\nAggSpecSum\x1a\x0c\n\nAggSpecVarB\x06\n\x04type\"\xdc\x02\n\x10\x41ggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12L\n\x11initial_groups_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x16\n\x0epreserve_empty\x18\x04 \x01(\x08\x12\x44\n\x0c\x61ggregations\x18\x05 \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x18\n\x10group_by_columns\x18\x06 \x03(\t\"\x9e\x08\n\x0b\x41ggregation\x12T\n\x07\x63olumns\x18\x01 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumnsH\x00\x12P\n\x05\x63ount\x18\x02 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountH\x00\x12Y\n\rfirst_row_key\x18\x03 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\x0clast_row_key\x18\x04 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\tpartition\x18\x05 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartitionH\x00\x12T\n\x07\x66ormula\x18\x06 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormulaH\x00\x12[\n\x0b\x63ount_where\x18\x07 \x01(\x0b\x32\x44.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhereH\x00\x1a\x63\n\x12\x41ggregationColumns\x12\x38\n\x04spec\x18\x01 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\'\n\x10\x41ggregationCount\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a=\n\x15\x41ggregationCountWhere\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x0f\n\x07\x66ilters\x18\x02 \x03(\t\x1a(\n\x11\x41ggregationRowKey\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1aM\n\x14\x41ggregationPartition\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12 \n\x18include_group_by_columns\x18\x02 \x01(\x08\x1aW\n\x12\x41ggregationFormula\x12\x41\n\nselectable\x18\x01 \x01(\x0b\x32-.io.deephaven.proto.backplane.grpc.SelectableB\x06\n\x04type\"\xe1\x01\n\x0eSortDescriptor\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x13\n\x0bis_absolute\x18\x02 \x01(\x08\x12R\n\tdirection\x18\x03 \x01(\x0e\x32?.io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection\"Q\n\rSortDirection\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x17\n\nDESCENDING\x10\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01\x12\r\n\tASCENDING\x10\x01\x12\x0b\n\x07REVERSE\x10\x02\"\xd8\x01\n\x10SortTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12@\n\x05sorts\x18\x03 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.SortDescriptor\"\xd7\x01\n\x12\x46ilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12=\n\x07\x66ilters\x18\x03 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xf9\x01\n\x0eSeekRowRequest\x12<\n\tsource_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x18\n\x0cstarting_row\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12>\n\nseek_value\x18\x04 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.Literal\x12\x13\n\x0binsensitive\x18\x05 \x01(\x08\x12\x10\n\x08\x63ontains\x18\x06 \x01(\x08\x12\x13\n\x0bis_backward\x18\x07 \x01(\x08\")\n\x0fSeekRowResponse\x12\x16\n\nresult_row\x18\x01 \x01(\x12\x42\x02\x30\x01\" \n\tReference\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\"\x91\x01\n\x07Literal\x12\x16\n\x0cstring_value\x18\x01 \x01(\tH\x00\x12\x16\n\x0c\x64ouble_value\x18\x02 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x03 \x01(\x08H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1d\n\x0fnano_time_value\x18\x05 \x01(\x12\x42\x02\x30\x01H\x00\x42\x07\n\x05value\"\x91\x01\n\x05Value\x12\x41\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.ReferenceH\x00\x12=\n\x07literal\x18\x02 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.LiteralH\x00\x42\x06\n\x04\x64\x61ta\"\xbc\x05\n\tCondition\x12>\n\x03\x61nd\x18\x01 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.AndConditionH\x00\x12<\n\x02or\x18\x02 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.OrConditionH\x00\x12>\n\x03not\x18\x03 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.NotConditionH\x00\x12\x46\n\x07\x63ompare\x18\x04 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.CompareConditionH\x00\x12<\n\x02in\x18\x05 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.InConditionH\x00\x12\x44\n\x06invoke\x18\x06 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.InvokeConditionH\x00\x12\x45\n\x07is_null\x18\x07 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.IsNullConditionH\x00\x12\x46\n\x07matches\x18\x08 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MatchesConditionH\x00\x12H\n\x08\x63ontains\x18\t \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.ContainsConditionH\x00\x12\x44\n\x06search\x18\n \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.SearchConditionH\x00\x42\x06\n\x04\x64\x61ta\"M\n\x0c\x41ndCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0bOrCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0cNotCondition\x12<\n\x06\x66ilter\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xac\x03\n\x10\x43ompareCondition\x12W\n\toperation\x18\x01 \x01(\x0e\x32\x44.io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation\x12L\n\x10\x63\x61se_sensitivity\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12\x35\n\x03lhs\x18\x03 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12\x35\n\x03rhs\x18\x04 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"\x82\x01\n\x10\x43ompareOperation\x12\r\n\tLESS_THAN\x10\x00\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x01\x12\x10\n\x0cGREATER_THAN\x10\x02\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x03\x12\n\n\x06\x45QUALS\x10\x04\x12\x0e\n\nNOT_EQUALS\x10\x05\"\x95\x02\n\x0bInCondition\x12\x38\n\x06target\x18\x01 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12<\n\ncandidates\x18\x02 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\x98\x01\n\x0fInvokeCondition\x12\x0e\n\x06method\x18\x01 \x01(\t\x12\x38\n\x06target\x18\x02 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12;\n\targuments\x18\x03 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"R\n\x0fIsNullCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\xf2\x01\n\x10MatchesCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\r\n\x05regex\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\xfb\x01\n\x11\x43ontainsCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\x15\n\rsearch_string\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"s\n\x0fSearchCondition\x12\x15\n\rsearch_string\x18\x01 \x01(\t\x12I\n\x13optional_references\x18\x02 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\x94\x01\n\x0e\x46lattenRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\x96\x01\n\x10MetaTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb4\x03\n\x19RunChartDownsampleRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0bpixel_count\x18\x03 \x01(\x05\x12Z\n\nzoom_range\x18\x04 \x01(\x0b\x32\x46.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange\x12\x15\n\rx_column_name\x18\x05 \x01(\t\x12\x16\n\x0ey_column_names\x18\x06 \x03(\t\x1as\n\tZoomRange\x12\x1f\n\x0emin_date_nanos\x18\x01 \x01(\x03\x42\x02\x30\x01H\x00\x88\x01\x01\x12\x1f\n\x0emax_date_nanos\x18\x02 \x01(\x03\x42\x02\x30\x01H\x01\x88\x01\x01\x42\x11\n\x0f_min_date_nanosB\x11\n\x0f_max_date_nanos\"\xe0\x05\n\x17\x43reateInputTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x0fsource_table_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReferenceH\x00\x12\x10\n\x06schema\x18\x03 \x01(\x0cH\x00\x12W\n\x04kind\x18\x04 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind\x1a\xbf\x03\n\x0eInputTableKind\x12}\n\x15in_memory_append_only\x18\x01 \x01(\x0b\x32\\.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnlyH\x00\x12{\n\x14in_memory_key_backed\x18\x02 \x01(\x0b\x32[.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBackedH\x00\x12`\n\x05\x62link\x18\x03 \x01(\x0b\x32O.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.BlinkH\x00\x1a\x14\n\x12InMemoryAppendOnly\x1a(\n\x11InMemoryKeyBacked\x12\x13\n\x0bkey_columns\x18\x01 \x03(\t\x1a\x07\n\x05\x42linkB\x06\n\x04kindB\x0c\n\ndefinition\"\x83\x02\n\x0eWhereInRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x10\n\x08inverted\x18\x04 \x01(\x08\x12\x18\n\x10\x63olumns_to_match\x18\x05 \x03(\t\"\xea\x01\n\x17\x43olumnStatisticsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x1f\n\x12unique_value_limit\x18\x04 \x01(\x05H\x00\x88\x01\x01\x42\x15\n\x13_unique_value_limit\"\xdd\x01\n\x0cSliceRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12$\n\x18\x66irst_position_inclusive\x18\x03 \x01(\x12\x42\x02\x30\x01\x12#\n\x17last_position_exclusive\x18\x04 \x01(\x12\x42\x02\x30\x01\"\xdb\x1a\n\x11\x42\x61tchTableRequest\x12K\n\x03ops\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation\x1a\xf8\x19\n\tOperation\x12K\n\x0b\x65mpty_table\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequestH\x00\x12I\n\ntime_table\x18\x02 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.TimeTableRequestH\x00\x12M\n\x0c\x64rop_columns\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequestH\x00\x12J\n\x06update\x18\x04 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0blazy_update\x18\x05 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12H\n\x04view\x18\x06 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0bupdate_view\x18\x07 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12J\n\x06select\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12S\n\x0fselect_distinct\x18\t \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequestH\x00\x12G\n\x06\x66ilter\x18\n \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.FilterTableRequestH\x00\x12`\n\x13unstructured_filter\x18\x0b \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequestH\x00\x12\x43\n\x04sort\x18\x0c \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.SortTableRequestH\x00\x12\x44\n\x04head\x18\r \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12\x44\n\x04tail\x18\x0e \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12I\n\x07head_by\x18\x0f \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12I\n\x07tail_by\x18\x10 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12\x44\n\x07ungroup\x18\x11 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.UngroupRequestH\x00\x12\x46\n\x05merge\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequestH\x00\x12S\n\x0f\x63ombo_aggregate\x18\x13 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequestH\x00\x12\x44\n\x07\x66latten\x18\x15 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.FlattenRequestH\x00\x12\\\n\x14run_chart_downsample\x18\x16 \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequestH\x00\x12O\n\ncross_join\x18\x17 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequestH\x00\x12S\n\x0cnatural_join\x18\x18 \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequestH\x00\x12O\n\nexact_join\x18\x19 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequestH\x00\x12M\n\tleft_join\x18\x1a \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequestH\x00\x12R\n\nas_of_join\x18\x1b \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequestB\x02\x18\x01H\x00\x12K\n\x0b\x66\x65tch_table\x18\x1c \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.FetchTableRequestH\x00\x12^\n\x15\x61pply_preview_columns\x18\x1e \x01(\x0b\x32=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequestH\x00\x12X\n\x12\x63reate_input_table\x18\x1f \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.CreateInputTableRequestH\x00\x12G\n\tupdate_by\x18 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.UpdateByRequestH\x00\x12\x45\n\x08where_in\x18! \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.WhereInRequestH\x00\x12O\n\raggregate_all\x18\" \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequestH\x00\x12H\n\taggregate\x18# \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.AggregateRequestH\x00\x12K\n\x08snapshot\x18$ \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequestH\x00\x12T\n\rsnapshot_when\x18% \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequestH\x00\x12I\n\nmeta_table\x18& \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MetaTableRequestH\x00\x12O\n\nrange_join\x18\' \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequestH\x00\x12\x43\n\x02\x61j\x18( \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12\x44\n\x03raj\x18) \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12W\n\x11\x63olumn_statistics\x18* \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequestH\x00\x12O\n\nmulti_join\x18+ \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequestH\x00\x12@\n\x05slice\x18, \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.SliceRequestH\x00\x42\x04\n\x02opJ\x04\x08\x14\x10\x15J\x04\x08\x1d\x10\x1e*b\n\x0f\x42\x61\x64\x44\x61taBehavior\x12#\n\x1f\x42\x41\x44_DATA_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\t\n\x05THROW\x10\x01\x12\t\n\x05RESET\x10\x02\x12\x08\n\x04SKIP\x10\x03\x12\n\n\x06POISON\x10\x04*t\n\x14UpdateByNullBehavior\x12\x1f\n\x1bNULL_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\x12\n\x0eNULL_DOMINATES\x10\x01\x12\x13\n\x0fVALUE_DOMINATES\x10\x02\x12\x12\n\x0eZERO_DOMINATES\x10\x03*\x1b\n\tNullValue\x12\x0e\n\nNULL_VALUE\x10\x00*2\n\x0f\x43\x61seSensitivity\x12\x0e\n\nMATCH_CASE\x10\x00\x12\x0f\n\x0bIGNORE_CASE\x10\x01*&\n\tMatchType\x12\x0b\n\x07REGULAR\x10\x00\x12\x0c\n\x08INVERTED\x10\x01\x32\xb9\x32\n\x0cTableService\x12\x91\x01\n GetExportedTableCreationResponse\x12).io.deephaven.proto.backplane.grpc.Ticket\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nFetchTable\x12\x34.io.deephaven.proto.backplane.grpc.FetchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x98\x01\n\x13\x41pplyPreviewColumns\x12=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nEmptyTable\x12\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tTimeTable\x12\x33.io.deephaven.proto.backplane.grpc.TimeTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0b\x44ropColumns\x12\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Update\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nLazyUpdate\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x04View\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nUpdateView\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Select\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x82\x01\n\x08UpdateBy\x12\x32.io.deephaven.proto.backplane.grpc.UpdateByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eSelectDistinct\x12\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x06\x46ilter\x12\x35.io.deephaven.proto.backplane.grpc.FilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x9b\x01\n\x12UnstructuredFilter\x12\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x7f\n\x04Sort\x12\x33.io.deephaven.proto.backplane.grpc.SortTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Head\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Tail\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06HeadBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06TailBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07Ungroup\x12\x31.io.deephaven.proto.backplane.grpc.UngroupRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0bMergeTables\x12\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x43rossJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x94\x01\n\x11NaturalJoinTables\x12;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x45xactJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eLeftJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x41sOfJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x85\x01\n\x08\x41jTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\tRajTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fMultiJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fRangeJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x43omboAggregate\x12\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x8a\x01\n\x0c\x41ggregateAll\x12\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tAggregate\x12\x33.io.deephaven.proto.backplane.grpc.AggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x87\x01\n\x08Snapshot\x12\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8f\x01\n\x0cSnapshotWhen\x12;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07\x46latten\x12\x31.io.deephaven.proto.backplane.grpc.FlattenRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x96\x01\n\x12RunChartDownsample\x12<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x92\x01\n\x10\x43reateInputTable\x12:.io.deephaven.proto.backplane.grpc.CreateInputTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07WhereIn\x12\x31.io.deephaven.proto.backplane.grpc.WhereInRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x05\x42\x61tch\x12\x34.io.deephaven.proto.backplane.grpc.BatchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x30\x01\x12\x99\x01\n\x14\x45xportedTableUpdates\x12>.io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest\x1a=.io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage\"\x00\x30\x01\x12r\n\x07SeekRow\x12\x31.io.deephaven.proto.backplane.grpc.SeekRowRequest\x1a\x32.io.deephaven.proto.backplane.grpc.SeekRowResponse\"\x00\x12\x84\x01\n\tMetaTable\x12\x33.io.deephaven.proto.backplane.grpc.MetaTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x99\x01\n\x17\x43omputeColumnStatistics\x12:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12|\n\x05Slice\x12/.io.deephaven.proto.backplane.grpc.SliceRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x42\x41H\x01P\x01Z;github.com/deephaven/deephaven-core/go/internal/proto/tableb\x06proto3')
+DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n deephaven_core/proto/table.proto\x12!io.deephaven.proto.backplane.grpc\x1a!deephaven_core/proto/ticket.proto\"l\n\x0eTableReference\x12;\n\x06ticket\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.TicketH\x00\x12\x16\n\x0c\x62\x61tch_offset\x18\x02 \x01(\x11H\x00\x42\x05\n\x03ref\"\xc6\x01\n\x1d\x45xportedTableCreationResponse\x12\x44\n\tresult_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07success\x18\x02 \x01(\x08\x12\x12\n\nerror_info\x18\x03 \x01(\t\x12\x15\n\rschema_header\x18\x04 \x01(\x0c\x12\x11\n\tis_static\x18\x05 \x01(\x08\x12\x10\n\x04size\x18\x06 \x01(\x12\x42\x02\x30\x01\"\x97\x01\n\x11\x46\x65tchTableRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\xa0\x01\n\x1a\x41pplyPreviewColumnsRequest\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12<\n\tresult_id\x18\x02 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\"\x1d\n\x1b\x45xportedTableUpdatesRequest\"\x8c\x01\n\x1a\x45xportedTableUpdateMessage\x12<\n\texport_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x1e\n\x16update_failure_message\x18\x03 \x01(\t\"c\n\x11\x45mptyTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x10\n\x04size\x18\x02 \x01(\x12\x42\x02\x30\x01\"\xef\x01\n\x10TimeTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x1e\n\x10start_time_nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1b\n\x11start_time_string\x18\x05 \x01(\tH\x00\x12\x1a\n\x0cperiod_nanos\x18\x03 \x01(\x12\x42\x02\x30\x01H\x01\x12\x17\n\rperiod_string\x18\x06 \x01(\tH\x01\x12\x13\n\x0b\x62link_table\x18\x04 \x01(\x08\x42\x0c\n\nstart_timeB\x08\n\x06period\"\xb1\x01\n\x15SelectOrUpdateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_specs\x18\x03 \x03(\t\"#\n\nSelectable\x12\r\n\x03raw\x18\x01 \x01(\tH\x00\x42\x06\n\x04type\"\x8c\x02\n\x0bMathContext\x12\x11\n\tprecision\x18\x01 \x01(\x11\x12R\n\rrounding_mode\x18\x02 \x01(\x0e\x32;.io.deephaven.proto.backplane.grpc.MathContext.RoundingMode\"\x95\x01\n\x0cRoundingMode\x12\x1f\n\x1bROUNDING_MODE_NOT_SPECIFIED\x10\x00\x12\x06\n\x02UP\x10\x01\x12\x08\n\x04\x44OWN\x10\x02\x12\x0b\n\x07\x43\x45ILING\x10\x03\x12\t\n\x05\x46LOOR\x10\x04\x12\x0b\n\x07HALF_UP\x10\x05\x12\r\n\tHALF_DOWN\x10\x06\x12\r\n\tHALF_EVEN\x10\x07\x12\x0f\n\x0bUNNECESSARY\x10\x08\"\xdb\x02\n\x13UpdateByWindowScale\x12[\n\x05ticks\x18\x01 \x01(\x0b\x32J.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTicksH\x00\x12Y\n\x04time\x18\x02 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.UpdateByWindowScale.UpdateByWindowTimeH\x00\x1a$\n\x13UpdateByWindowTicks\x12\r\n\x05ticks\x18\x01 \x01(\x01\x1a^\n\x12UpdateByWindowTime\x12\x0e\n\x06\x63olumn\x18\x01 \x01(\t\x12\x13\n\x05nanos\x18\x02 \x01(\x12\x42\x02\x30\x01H\x00\x12\x19\n\x0f\x64uration_string\x18\x03 \x01(\tH\x00\x42\x08\n\x06windowB\x06\n\x04type\"\xe1\x03\n\x11UpdateByEmOptions\x12I\n\ron_null_value\x18\x01 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_nan_value\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12H\n\x0con_null_time\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12R\n\x16on_negative_delta_time\x18\x04 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12N\n\x12on_zero_delta_time\x18\x05 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.BadDataBehavior\x12I\n\x11\x62ig_value_context\x18\x06 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.MathContext\"f\n\x14UpdateByDeltaOptions\x12N\n\rnull_behavior\x18\x01 \x01(\x0e\x32\x37.io.deephaven.proto.backplane.grpc.UpdateByNullBehavior\"\x88<\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\xe1\x35\n\x11UpdateByOperation\x12\x65\n\x06\x63olumn\x18\x01 \x01(\x0b\x32S.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumnH\x00\x1a\xdc\x34\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\xc4\x33\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\x12\x99\x01\n\x13rolling_count_where\x18\x16 \x01(\x0b\x32z.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByRollingCountWhereH\x00\x12\x94\x01\n\x0b\x63ount_where\x18\x17 \x01(\x0b\x32}.io.deephaven.proto.backplane.grpc.UpdateByRequest.UpdateByOperation.UpdateByColumn.UpdateBySpec.UpdateByCumulativeCountWhereH\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(\t\x1a\xef\x01\n\x19UpdateByRollingCountWhere\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\rresult_column\x18\x03 \x01(\t\x12\x0f\n\x07\x66ilters\x18\x04 \x03(\t\x1a\x46\n\x1cUpdateByCumulativeCountWhere\x12\x15\n\rresult_column\x18\x01 \x01(\t\x12\x0f\n\x07\x66ilters\x18\x02 \x03(\tB\x06\n\x04typeB\x06\n\x04type\"\xb1\x01\n\x15SelectDistinctRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xae\x01\n\x12\x44ropColumnsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x0c\x63olumn_names\x18\x03 \x03(\t\"\xb5\x01\n\x1eUnstructuredFilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07\x66ilters\x18\x03 \x03(\t\"\xad\x01\n\x11HeadOrTailRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\"\xce\x01\n\x13HeadOrTailByRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x14\n\x08num_rows\x18\x03 \x01(\x12\x42\x02\x30\x01\x12\x1d\n\x15group_by_column_specs\x18\x04 \x03(\t\"\xc3\x01\n\x0eUngroupRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x11\n\tnull_fill\x18\x03 \x01(\x08\x12\x1a\n\x12\x63olumns_to_ungroup\x18\x04 \x03(\t\"\xad\x01\n\x12MergeTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x45\n\nsource_ids\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x12\n\nkey_column\x18\x03 \x01(\t\"\x9a\x01\n\x14SnapshotTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb1\x02\n\x18SnapshotWhenTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07\x62\x61se_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x45\n\ntrigger_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x0f\n\x07initial\x18\x04 \x01(\x08\x12\x13\n\x0bincremental\x18\x05 \x01(\x08\x12\x0f\n\x07history\x18\x06 \x01(\x08\x12\x15\n\rstamp_columns\x18\x07 \x03(\t\"\xa7\x02\n\x16\x43rossJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\x14\n\x0creserve_bits\x18\x06 \x01(\x05\"\x93\x02\n\x18NaturalJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x91\x02\n\x16\x45xactJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\x90\x02\n\x15LeftJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\"\xd1\x03\n\x15\x41sOfJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x04 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x05 \x03(\t\x12\\\n\x10\x61s_of_match_rule\x18\x07 \x01(\x0e\x32\x42.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest.MatchRule\"]\n\tMatchRule\x12\x13\n\x0fLESS_THAN_EQUAL\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12GREATER_THAN_EQUAL\x10\x02\x12\x10\n\x0cGREATER_THAN\x10\x03\x1a\x02\x18\x01:\x02\x18\x01\"\xa6\x02\n\x12\x41jRajTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x14\n\x0c\x61s_of_column\x18\x05 \x01(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x06 \x03(\t\"\x88\x01\n\x0eMultiJoinInput\x12\x44\n\tsource_id\x18\x01 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x18\n\x10\x63olumns_to_match\x18\x02 \x03(\t\x12\x16\n\x0e\x63olumns_to_add\x18\x03 \x03(\t\"\xa4\x01\n\x16MultiJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x11multi_join_inputs\x18\x02 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.MultiJoinInput\"\xe0\x06\n\x16RangeJoinTablesRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x1b\n\x13\x65xact_match_columns\x18\x04 \x03(\t\x12\x19\n\x11left_start_column\x18\x05 \x01(\t\x12\x62\n\x10range_start_rule\x18\x06 \x01(\x0e\x32H.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeStartRule\x12\x1a\n\x12right_range_column\x18\x07 \x01(\t\x12^\n\x0erange_end_rule\x18\x08 \x01(\x0e\x32\x46.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest.RangeEndRule\x12\x17\n\x0fleft_end_column\x18\t \x01(\t\x12\x44\n\x0c\x61ggregations\x18\n \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x13\n\x0brange_match\x18\x0b \x01(\t\"v\n\x0eRangeStartRule\x12\x15\n\x11START_UNSPECIFIED\x10\x00\x12\r\n\tLESS_THAN\x10\x01\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x02\x12&\n\"LESS_THAN_OR_EQUAL_ALLOW_PRECEDING\x10\x03\"{\n\x0cRangeEndRule\x12\x13\n\x0f\x45ND_UNSPECIFIED\x10\x00\x12\x10\n\x0cGREATER_THAN\x10\x01\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x02\x12)\n%GREATER_THAN_OR_EQUAL_ALLOW_FOLLOWING\x10\x03\"\xfe\x04\n\x15\x43omboAggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12V\n\naggregates\x18\x03 \x03(\x0b\x32\x42.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.Aggregate\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\x12\x13\n\x0b\x66orce_combo\x18\x05 \x01(\x08\x1a\xad\x01\n\tAggregate\x12N\n\x04type\x18\x01 \x01(\x0e\x32@.io.deephaven.proto.backplane.grpc.ComboAggregateRequest.AggType\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x12\n\npercentile\x18\x04 \x01(\x01\x12\x12\n\navg_median\x18\x05 \x01(\x08\"\xa5\x01\n\x07\x41ggType\x12\x07\n\x03SUM\x10\x00\x12\x0b\n\x07\x41\x42S_SUM\x10\x01\x12\t\n\x05GROUP\x10\x02\x12\x07\n\x03\x41VG\x10\x03\x12\t\n\x05\x43OUNT\x10\x04\x12\t\n\x05\x46IRST\x10\x05\x12\x08\n\x04LAST\x10\x06\x12\x07\n\x03MIN\x10\x07\x12\x07\n\x03MAX\x10\x08\x12\n\n\x06MEDIAN\x10\t\x12\x0e\n\nPERCENTILE\x10\n\x12\x07\n\x03STD\x10\x0b\x12\x07\n\x03VAR\x10\x0c\x12\x10\n\x0cWEIGHTED_AVG\x10\r:\x02\x18\x01\"\xed\x01\n\x13\x41ggregateAllRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x38\n\x04spec\x18\x03 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x18\n\x10group_by_columns\x18\x04 \x03(\t\"\xd7\x17\n\x07\x41ggSpec\x12K\n\x07\x61\x62s_sum\x18\x01 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAbsSumH\x00\x12i\n\x16\x61pproximate_percentile\x18\x02 \x01(\x0b\x32G.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecApproximatePercentileH\x00\x12\x44\n\x03\x61vg\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecAvgH\x00\x12Y\n\x0e\x63ount_distinct\x18\x04 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecCountDistinctH\x00\x12N\n\x08\x64istinct\x18\x05 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecDistinctH\x00\x12H\n\x05\x66irst\x18\x06 \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFirstH\x00\x12L\n\x07\x66ormula\x18\x07 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFormulaH\x00\x12J\n\x06\x66reeze\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecFreezeH\x00\x12H\n\x05group\x18\t \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecGroupH\x00\x12\x46\n\x04last\x18\n \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecLastH\x00\x12\x44\n\x03max\x18\x0b \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMaxH\x00\x12J\n\x06median\x18\x0c \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMedianH\x00\x12\x44\n\x03min\x18\r \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecMinH\x00\x12R\n\npercentile\x18\x0e \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecPercentileH\x00\x12P\n\x0csorted_first\x18\x0f \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12O\n\x0bsorted_last\x18\x10 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedH\x00\x12\x44\n\x03std\x18\x11 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecStdH\x00\x12\x44\n\x03sum\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSumH\x00\x12M\n\x08t_digest\x18\x13 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecTDigestH\x00\x12J\n\x06unique\x18\x14 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecUniqueH\x00\x12R\n\x0cweighted_avg\x18\x15 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12R\n\x0cweighted_sum\x18\x16 \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecWeightedH\x00\x12\x44\n\x03var\x18\x17 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecVarH\x00\x1a\\\n\x1c\x41ggSpecApproximatePercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x18\n\x0b\x63ompression\x18\x02 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a+\n\x14\x41ggSpecCountDistinct\x12\x13\n\x0b\x63ount_nulls\x18\x01 \x01(\x08\x1a(\n\x0f\x41ggSpecDistinct\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x1a\x36\n\x0e\x41ggSpecFormula\x12\x0f\n\x07\x66ormula\x18\x01 \x01(\t\x12\x13\n\x0bparam_token\x18\x02 \x01(\t\x1a/\n\rAggSpecMedian\x12\x1e\n\x16\x61verage_evenly_divided\x18\x01 \x01(\x08\x1aG\n\x11\x41ggSpecPercentile\x12\x12\n\npercentile\x18\x01 \x01(\x01\x12\x1e\n\x16\x61verage_evenly_divided\x18\x02 \x01(\x08\x1a`\n\rAggSpecSorted\x12O\n\x07\x63olumns\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecSortedColumn\x1a*\n\x13\x41ggSpecSortedColumn\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a:\n\x0e\x41ggSpecTDigest\x12\x18\n\x0b\x63ompression\x18\x01 \x01(\x01H\x00\x88\x01\x01\x42\x0e\n\x0c_compression\x1a\x88\x01\n\rAggSpecUnique\x12\x15\n\rinclude_nulls\x18\x01 \x01(\x08\x12`\n\x13non_unique_sentinel\x18\x02 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.AggSpec.AggSpecNonUniqueSentinel\x1a\xb5\x02\n\x18\x41ggSpecNonUniqueSentinel\x12\x42\n\nnull_value\x18\x01 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.NullValueH\x00\x12\x16\n\x0cstring_value\x18\x02 \x01(\tH\x00\x12\x13\n\tint_value\x18\x03 \x01(\x11H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x15\n\x0b\x66loat_value\x18\x05 \x01(\x02H\x00\x12\x16\n\x0c\x64ouble_value\x18\x06 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x07 \x01(\x08H\x00\x12\x14\n\nbyte_value\x18\x08 \x01(\x11H\x00\x12\x15\n\x0bshort_value\x18\t \x01(\x11H\x00\x12\x14\n\nchar_value\x18\n \x01(\x11H\x00\x42\x06\n\x04type\x1a(\n\x0f\x41ggSpecWeighted\x12\x15\n\rweight_column\x18\x01 \x01(\t\x1a\x0f\n\rAggSpecAbsSum\x1a\x0c\n\nAggSpecAvg\x1a\x0e\n\x0c\x41ggSpecFirst\x1a\x0f\n\rAggSpecFreeze\x1a\x0e\n\x0c\x41ggSpecGroup\x1a\r\n\x0b\x41ggSpecLast\x1a\x0c\n\nAggSpecMax\x1a\x0c\n\nAggSpecMin\x1a\x0c\n\nAggSpecStd\x1a\x0c\n\nAggSpecSum\x1a\x0c\n\nAggSpecVarB\x06\n\x04type\"\xdc\x02\n\x10\x41ggregateRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12L\n\x11initial_groups_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x16\n\x0epreserve_empty\x18\x04 \x01(\x08\x12\x44\n\x0c\x61ggregations\x18\x05 \x03(\x0b\x32..io.deephaven.proto.backplane.grpc.Aggregation\x12\x18\n\x10group_by_columns\x18\x06 \x03(\t\"\x9e\x08\n\x0b\x41ggregation\x12T\n\x07\x63olumns\x18\x01 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationColumnsH\x00\x12P\n\x05\x63ount\x18\x02 \x01(\x0b\x32?.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountH\x00\x12Y\n\rfirst_row_key\x18\x03 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\x0clast_row_key\x18\x04 \x01(\x0b\x32@.io.deephaven.proto.backplane.grpc.Aggregation.AggregationRowKeyH\x00\x12X\n\tpartition\x18\x05 \x01(\x0b\x32\x43.io.deephaven.proto.backplane.grpc.Aggregation.AggregationPartitionH\x00\x12T\n\x07\x66ormula\x18\x06 \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.Aggregation.AggregationFormulaH\x00\x12[\n\x0b\x63ount_where\x18\x07 \x01(\x0b\x32\x44.io.deephaven.proto.backplane.grpc.Aggregation.AggregationCountWhereH\x00\x1a\x63\n\x12\x41ggregationColumns\x12\x38\n\x04spec\x18\x01 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.AggSpec\x12\x13\n\x0bmatch_pairs\x18\x02 \x03(\t\x1a\'\n\x10\x41ggregationCount\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1a=\n\x15\x41ggregationCountWhere\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x0f\n\x07\x66ilters\x18\x02 \x03(\t\x1a(\n\x11\x41ggregationRowKey\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x1aM\n\x14\x41ggregationPartition\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12 \n\x18include_group_by_columns\x18\x02 \x01(\x08\x1aW\n\x12\x41ggregationFormula\x12\x41\n\nselectable\x18\x01 \x01(\x0b\x32-.io.deephaven.proto.backplane.grpc.SelectableB\x06\n\x04type\"\xe1\x01\n\x0eSortDescriptor\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\x12\x13\n\x0bis_absolute\x18\x02 \x01(\x08\x12R\n\tdirection\x18\x03 \x01(\x0e\x32?.io.deephaven.proto.backplane.grpc.SortDescriptor.SortDirection\"Q\n\rSortDirection\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x17\n\nDESCENDING\x10\xff\xff\xff\xff\xff\xff\xff\xff\xff\x01\x12\r\n\tASCENDING\x10\x01\x12\x0b\n\x07REVERSE\x10\x02\"\xd8\x01\n\x10SortTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12@\n\x05sorts\x18\x03 \x03(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.SortDescriptor\"\xd7\x01\n\x12\x46ilterTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12=\n\x07\x66ilters\x18\x03 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xf9\x01\n\x0eSeekRowRequest\x12<\n\tsource_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x18\n\x0cstarting_row\x18\x02 \x01(\x12\x42\x02\x30\x01\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12>\n\nseek_value\x18\x04 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.Literal\x12\x13\n\x0binsensitive\x18\x05 \x01(\x08\x12\x10\n\x08\x63ontains\x18\x06 \x01(\x08\x12\x13\n\x0bis_backward\x18\x07 \x01(\x08\")\n\x0fSeekRowResponse\x12\x16\n\nresult_row\x18\x01 \x01(\x12\x42\x02\x30\x01\" \n\tReference\x12\x13\n\x0b\x63olumn_name\x18\x01 \x01(\t\"\x91\x01\n\x07Literal\x12\x16\n\x0cstring_value\x18\x01 \x01(\tH\x00\x12\x16\n\x0c\x64ouble_value\x18\x02 \x01(\x01H\x00\x12\x14\n\nbool_value\x18\x03 \x01(\x08H\x00\x12\x18\n\nlong_value\x18\x04 \x01(\x12\x42\x02\x30\x01H\x00\x12\x1d\n\x0fnano_time_value\x18\x05 \x01(\x12\x42\x02\x30\x01H\x00\x42\x07\n\x05value\"\x91\x01\n\x05Value\x12\x41\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.ReferenceH\x00\x12=\n\x07literal\x18\x02 \x01(\x0b\x32*.io.deephaven.proto.backplane.grpc.LiteralH\x00\x42\x06\n\x04\x64\x61ta\"\xbc\x05\n\tCondition\x12>\n\x03\x61nd\x18\x01 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.AndConditionH\x00\x12<\n\x02or\x18\x02 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.OrConditionH\x00\x12>\n\x03not\x18\x03 \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.NotConditionH\x00\x12\x46\n\x07\x63ompare\x18\x04 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.CompareConditionH\x00\x12<\n\x02in\x18\x05 \x01(\x0b\x32..io.deephaven.proto.backplane.grpc.InConditionH\x00\x12\x44\n\x06invoke\x18\x06 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.InvokeConditionH\x00\x12\x45\n\x07is_null\x18\x07 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.IsNullConditionH\x00\x12\x46\n\x07matches\x18\x08 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MatchesConditionH\x00\x12H\n\x08\x63ontains\x18\t \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.ContainsConditionH\x00\x12\x44\n\x06search\x18\n \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.SearchConditionH\x00\x42\x06\n\x04\x64\x61ta\"M\n\x0c\x41ndCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0bOrCondition\x12=\n\x07\x66ilters\x18\x01 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"L\n\x0cNotCondition\x12<\n\x06\x66ilter\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Condition\"\xac\x03\n\x10\x43ompareCondition\x12W\n\toperation\x18\x01 \x01(\x0e\x32\x44.io.deephaven.proto.backplane.grpc.CompareCondition.CompareOperation\x12L\n\x10\x63\x61se_sensitivity\x18\x02 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12\x35\n\x03lhs\x18\x03 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12\x35\n\x03rhs\x18\x04 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"\x82\x01\n\x10\x43ompareOperation\x12\r\n\tLESS_THAN\x10\x00\x12\x16\n\x12LESS_THAN_OR_EQUAL\x10\x01\x12\x10\n\x0cGREATER_THAN\x10\x02\x12\x19\n\x15GREATER_THAN_OR_EQUAL\x10\x03\x12\n\n\x06\x45QUALS\x10\x04\x12\x0e\n\nNOT_EQUALS\x10\x05\"\x95\x02\n\x0bInCondition\x12\x38\n\x06target\x18\x01 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12<\n\ncandidates\x18\x02 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\x98\x01\n\x0fInvokeCondition\x12\x0e\n\x06method\x18\x01 \x01(\t\x12\x38\n\x06target\x18\x02 \x01(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\x12;\n\targuments\x18\x03 \x03(\x0b\x32(.io.deephaven.proto.backplane.grpc.Value\"R\n\x0fIsNullCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\xf2\x01\n\x10MatchesCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\r\n\x05regex\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"\xfb\x01\n\x11\x43ontainsCondition\x12?\n\treference\x18\x01 \x01(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\x12\x15\n\rsearch_string\x18\x02 \x01(\t\x12L\n\x10\x63\x61se_sensitivity\x18\x03 \x01(\x0e\x32\x32.io.deephaven.proto.backplane.grpc.CaseSensitivity\x12@\n\nmatch_type\x18\x04 \x01(\x0e\x32,.io.deephaven.proto.backplane.grpc.MatchType\"s\n\x0fSearchCondition\x12\x15\n\rsearch_string\x18\x01 \x01(\t\x12I\n\x13optional_references\x18\x02 \x03(\x0b\x32,.io.deephaven.proto.backplane.grpc.Reference\"\x94\x01\n\x0e\x46lattenRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\x96\x01\n\x10MetaTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\"\xb4\x03\n\x19RunChartDownsampleRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0bpixel_count\x18\x03 \x01(\x05\x12Z\n\nzoom_range\x18\x04 \x01(\x0b\x32\x46.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest.ZoomRange\x12\x15\n\rx_column_name\x18\x05 \x01(\t\x12\x16\n\x0ey_column_names\x18\x06 \x03(\t\x1as\n\tZoomRange\x12\x1f\n\x0emin_date_nanos\x18\x01 \x01(\x03\x42\x02\x30\x01H\x00\x88\x01\x01\x12\x1f\n\x0emax_date_nanos\x18\x02 \x01(\x03\x42\x02\x30\x01H\x01\x88\x01\x01\x42\x11\n\x0f_min_date_nanosB\x11\n\x0f_max_date_nanos\"\xe0\x05\n\x17\x43reateInputTableRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12L\n\x0fsource_table_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReferenceH\x00\x12\x10\n\x06schema\x18\x03 \x01(\x0cH\x00\x12W\n\x04kind\x18\x04 \x01(\x0b\x32I.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind\x1a\xbf\x03\n\x0eInputTableKind\x12}\n\x15in_memory_append_only\x18\x01 \x01(\x0b\x32\\.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryAppendOnlyH\x00\x12{\n\x14in_memory_key_backed\x18\x02 \x01(\x0b\x32[.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.InMemoryKeyBackedH\x00\x12`\n\x05\x62link\x18\x03 \x01(\x0b\x32O.io.deephaven.proto.backplane.grpc.CreateInputTableRequest.InputTableKind.BlinkH\x00\x1a\x14\n\x12InMemoryAppendOnly\x1a(\n\x11InMemoryKeyBacked\x12\x13\n\x0bkey_columns\x18\x01 \x03(\t\x1a\x07\n\x05\x42linkB\x06\n\x04kindB\x0c\n\ndefinition\"\x83\x02\n\x0eWhereInRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x42\n\x07left_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x43\n\x08right_id\x18\x03 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x10\n\x08inverted\x18\x04 \x01(\x08\x12\x18\n\x10\x63olumns_to_match\x18\x05 \x03(\t\"\xea\x01\n\x17\x43olumnStatisticsRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12\x13\n\x0b\x63olumn_name\x18\x03 \x01(\t\x12\x1f\n\x12unique_value_limit\x18\x04 \x01(\x05H\x00\x88\x01\x01\x42\x15\n\x13_unique_value_limit\"\xdd\x01\n\x0cSliceRequest\x12<\n\tresult_id\x18\x01 \x01(\x0b\x32).io.deephaven.proto.backplane.grpc.Ticket\x12\x44\n\tsource_id\x18\x02 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.TableReference\x12$\n\x18\x66irst_position_inclusive\x18\x03 \x01(\x12\x42\x02\x30\x01\x12#\n\x17last_position_exclusive\x18\x04 \x01(\x12\x42\x02\x30\x01\"\xdb\x1a\n\x11\x42\x61tchTableRequest\x12K\n\x03ops\x18\x01 \x03(\x0b\x32>.io.deephaven.proto.backplane.grpc.BatchTableRequest.Operation\x1a\xf8\x19\n\tOperation\x12K\n\x0b\x65mpty_table\x18\x01 \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequestH\x00\x12I\n\ntime_table\x18\x02 \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.TimeTableRequestH\x00\x12M\n\x0c\x64rop_columns\x18\x03 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequestH\x00\x12J\n\x06update\x18\x04 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0blazy_update\x18\x05 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12H\n\x04view\x18\x06 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12O\n\x0bupdate_view\x18\x07 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12J\n\x06select\x18\x08 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequestH\x00\x12S\n\x0fselect_distinct\x18\t \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequestH\x00\x12G\n\x06\x66ilter\x18\n \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.FilterTableRequestH\x00\x12`\n\x13unstructured_filter\x18\x0b \x01(\x0b\x32\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequestH\x00\x12\x43\n\x04sort\x18\x0c \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.SortTableRequestH\x00\x12\x44\n\x04head\x18\r \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12\x44\n\x04tail\x18\x0e \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequestH\x00\x12I\n\x07head_by\x18\x0f \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12I\n\x07tail_by\x18\x10 \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequestH\x00\x12\x44\n\x07ungroup\x18\x11 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.UngroupRequestH\x00\x12\x46\n\x05merge\x18\x12 \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequestH\x00\x12S\n\x0f\x63ombo_aggregate\x18\x13 \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequestH\x00\x12\x44\n\x07\x66latten\x18\x15 \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.FlattenRequestH\x00\x12\\\n\x14run_chart_downsample\x18\x16 \x01(\x0b\x32<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequestH\x00\x12O\n\ncross_join\x18\x17 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequestH\x00\x12S\n\x0cnatural_join\x18\x18 \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequestH\x00\x12O\n\nexact_join\x18\x19 \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequestH\x00\x12M\n\tleft_join\x18\x1a \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequestH\x00\x12R\n\nas_of_join\x18\x1b \x01(\x0b\x32\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequestB\x02\x18\x01H\x00\x12K\n\x0b\x66\x65tch_table\x18\x1c \x01(\x0b\x32\x34.io.deephaven.proto.backplane.grpc.FetchTableRequestH\x00\x12^\n\x15\x61pply_preview_columns\x18\x1e \x01(\x0b\x32=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequestH\x00\x12X\n\x12\x63reate_input_table\x18\x1f \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.CreateInputTableRequestH\x00\x12G\n\tupdate_by\x18 \x01(\x0b\x32\x32.io.deephaven.proto.backplane.grpc.UpdateByRequestH\x00\x12\x45\n\x08where_in\x18! \x01(\x0b\x32\x31.io.deephaven.proto.backplane.grpc.WhereInRequestH\x00\x12O\n\raggregate_all\x18\" \x01(\x0b\x32\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequestH\x00\x12H\n\taggregate\x18# \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.AggregateRequestH\x00\x12K\n\x08snapshot\x18$ \x01(\x0b\x32\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequestH\x00\x12T\n\rsnapshot_when\x18% \x01(\x0b\x32;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequestH\x00\x12I\n\nmeta_table\x18& \x01(\x0b\x32\x33.io.deephaven.proto.backplane.grpc.MetaTableRequestH\x00\x12O\n\nrange_join\x18\' \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequestH\x00\x12\x43\n\x02\x61j\x18( \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12\x44\n\x03raj\x18) \x01(\x0b\x32\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequestH\x00\x12W\n\x11\x63olumn_statistics\x18* \x01(\x0b\x32:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequestH\x00\x12O\n\nmulti_join\x18+ \x01(\x0b\x32\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequestH\x00\x12@\n\x05slice\x18, \x01(\x0b\x32/.io.deephaven.proto.backplane.grpc.SliceRequestH\x00\x42\x04\n\x02opJ\x04\x08\x14\x10\x15J\x04\x08\x1d\x10\x1e*b\n\x0f\x42\x61\x64\x44\x61taBehavior\x12#\n\x1f\x42\x41\x44_DATA_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\t\n\x05THROW\x10\x01\x12\t\n\x05RESET\x10\x02\x12\x08\n\x04SKIP\x10\x03\x12\n\n\x06POISON\x10\x04*t\n\x14UpdateByNullBehavior\x12\x1f\n\x1bNULL_BEHAVIOR_NOT_SPECIFIED\x10\x00\x12\x12\n\x0eNULL_DOMINATES\x10\x01\x12\x13\n\x0fVALUE_DOMINATES\x10\x02\x12\x12\n\x0eZERO_DOMINATES\x10\x03*\x1b\n\tNullValue\x12\x0e\n\nNULL_VALUE\x10\x00*2\n\x0f\x43\x61seSensitivity\x12\x0e\n\nMATCH_CASE\x10\x00\x12\x0f\n\x0bIGNORE_CASE\x10\x01*&\n\tMatchType\x12\x0b\n\x07REGULAR\x10\x00\x12\x0c\n\x08INVERTED\x10\x01\x32\xb9\x32\n\x0cTableService\x12\x91\x01\n GetExportedTableCreationResponse\x12).io.deephaven.proto.backplane.grpc.Ticket\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nFetchTable\x12\x34.io.deephaven.proto.backplane.grpc.FetchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x98\x01\n\x13\x41pplyPreviewColumns\x12=.io.deephaven.proto.backplane.grpc.ApplyPreviewColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\nEmptyTable\x12\x34.io.deephaven.proto.backplane.grpc.EmptyTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tTimeTable\x12\x33.io.deephaven.proto.backplane.grpc.TimeTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0b\x44ropColumns\x12\x35.io.deephaven.proto.backplane.grpc.DropColumnsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Update\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nLazyUpdate\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x04View\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8a\x01\n\nUpdateView\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\x06Select\x12\x38.io.deephaven.proto.backplane.grpc.SelectOrUpdateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x82\x01\n\x08UpdateBy\x12\x32.io.deephaven.proto.backplane.grpc.UpdateByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eSelectDistinct\x12\x38.io.deephaven.proto.backplane.grpc.SelectDistinctRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x06\x46ilter\x12\x35.io.deephaven.proto.backplane.grpc.FilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x9b\x01\n\x12UnstructuredFilter\x12\x41.io.deephaven.proto.backplane.grpc.UnstructuredFilterTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x7f\n\x04Sort\x12\x33.io.deephaven.proto.backplane.grpc.SortTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Head\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x04Tail\x12\x34.io.deephaven.proto.backplane.grpc.HeadOrTailRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06HeadBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\x06TailBy\x12\x36.io.deephaven.proto.backplane.grpc.HeadOrTailByRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07Ungroup\x12\x31.io.deephaven.proto.backplane.grpc.UngroupRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x88\x01\n\x0bMergeTables\x12\x35.io.deephaven.proto.backplane.grpc.MergeTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x43rossJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.CrossJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x94\x01\n\x11NaturalJoinTables\x12;.io.deephaven.proto.backplane.grpc.NaturalJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0f\x45xactJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.ExactJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8e\x01\n\x0eLeftJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.LeftJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x41sOfJoinTables\x12\x38.io.deephaven.proto.backplane.grpc.AsOfJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x85\x01\n\x08\x41jTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x86\x01\n\tRajTables\x12\x35.io.deephaven.proto.backplane.grpc.AjRajTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fMultiJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.MultiJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x90\x01\n\x0fRangeJoinTables\x12\x39.io.deephaven.proto.backplane.grpc.RangeJoinTablesRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x91\x01\n\x0e\x43omboAggregate\x12\x38.io.deephaven.proto.backplane.grpc.ComboAggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x03\x88\x02\x01\x12\x8a\x01\n\x0c\x41ggregateAll\x12\x36.io.deephaven.proto.backplane.grpc.AggregateAllRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x84\x01\n\tAggregate\x12\x33.io.deephaven.proto.backplane.grpc.AggregateRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x87\x01\n\x08Snapshot\x12\x37.io.deephaven.proto.backplane.grpc.SnapshotTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x8f\x01\n\x0cSnapshotWhen\x12;.io.deephaven.proto.backplane.grpc.SnapshotWhenTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07\x46latten\x12\x31.io.deephaven.proto.backplane.grpc.FlattenRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x96\x01\n\x12RunChartDownsample\x12<.io.deephaven.proto.backplane.grpc.RunChartDownsampleRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x92\x01\n\x10\x43reateInputTable\x12:.io.deephaven.proto.backplane.grpc.CreateInputTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x80\x01\n\x07WhereIn\x12\x31.io.deephaven.proto.backplane.grpc.WhereInRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x83\x01\n\x05\x42\x61tch\x12\x34.io.deephaven.proto.backplane.grpc.BatchTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x30\x01\x12\x99\x01\n\x14\x45xportedTableUpdates\x12>.io.deephaven.proto.backplane.grpc.ExportedTableUpdatesRequest\x1a=.io.deephaven.proto.backplane.grpc.ExportedTableUpdateMessage\"\x00\x30\x01\x12r\n\x07SeekRow\x12\x31.io.deephaven.proto.backplane.grpc.SeekRowRequest\x1a\x32.io.deephaven.proto.backplane.grpc.SeekRowResponse\"\x00\x12\x84\x01\n\tMetaTable\x12\x33.io.deephaven.proto.backplane.grpc.MetaTableRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12\x99\x01\n\x17\x43omputeColumnStatistics\x12:.io.deephaven.proto.backplane.grpc.ColumnStatisticsRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x12|\n\x05Slice\x12/.io.deephaven.proto.backplane.grpc.SliceRequest\x1a@.io.deephaven.proto.backplane.grpc.ExportedTableCreationResponse\"\x00\x42\x41H\x01P\x01Z;github.com/deephaven/deephaven-core/go/internal/proto/tableb\x06proto3')
_globals = globals()
_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals)
@@ -69,16 +69,16 @@
_globals['_TABLESERVICE'].methods_by_name['AsOfJoinTables']._serialized_options = b'\210\002\001'
_globals['_TABLESERVICE'].methods_by_name['ComboAggregate']._loaded_options = None
_globals['_TABLESERVICE'].methods_by_name['ComboAggregate']._serialized_options = b'\210\002\001'
- _globals['_BADDATABEHAVIOR']._serialized_start=29321
- _globals['_BADDATABEHAVIOR']._serialized_end=29419
- _globals['_UPDATEBYNULLBEHAVIOR']._serialized_start=29421
- _globals['_UPDATEBYNULLBEHAVIOR']._serialized_end=29537
- _globals['_NULLVALUE']._serialized_start=29539
- _globals['_NULLVALUE']._serialized_end=29566
- _globals['_CASESENSITIVITY']._serialized_start=29568
- _globals['_CASESENSITIVITY']._serialized_end=29618
- _globals['_MATCHTYPE']._serialized_start=29620
- _globals['_MATCHTYPE']._serialized_end=29658
+ _globals['_BADDATABEHAVIOR']._serialized_start=29942
+ _globals['_BADDATABEHAVIOR']._serialized_end=30040
+ _globals['_UPDATEBYNULLBEHAVIOR']._serialized_start=30042
+ _globals['_UPDATEBYNULLBEHAVIOR']._serialized_end=30158
+ _globals['_NULLVALUE']._serialized_start=30160
+ _globals['_NULLVALUE']._serialized_end=30187
+ _globals['_CASESENSITIVITY']._serialized_start=30189
+ _globals['_CASESENSITIVITY']._serialized_end=30239
+ _globals['_MATCHTYPE']._serialized_start=30241
+ _globals['_MATCHTYPE']._serialized_end=30279
_globals['_TABLEREFERENCE']._serialized_start=106
_globals['_TABLEREFERENCE']._serialized_end=214
_globals['_EXPORTEDTABLECREATIONRESPONSE']._serialized_start=217
@@ -114,241 +114,245 @@
_globals['_UPDATEBYDELTAOPTIONS']._serialized_start=2573
_globals['_UPDATEBYDELTAOPTIONS']._serialized_end=2675
_globals['_UPDATEBYREQUEST']._serialized_start=2678
- _globals['_UPDATEBYREQUEST']._serialized_end=9745
+ _globals['_UPDATEBYREQUEST']._serialized_end=10366
_globals['_UPDATEBYREQUEST_UPDATEBYOPTIONS']._serialized_start=3031
_globals['_UPDATEBYREQUEST_UPDATEBYOPTIONS']._serialized_end=3482
_globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_start=3485
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_end=9745
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION']._serialized_end=10366
_globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_start=3610
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_end=9737
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN']._serialized_end=10358
_globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_start=3762
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_end=9737
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_start=6665
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_end=6688
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_start=6690
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_end=6713
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_start=6715
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_end=6738
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_start=6740
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_end=6767
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_start=6769
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_end=6783
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_start=6786
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_end=6948
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_start=6951
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_end=7113
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_start=7116
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_end=7280
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_start=7283
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_end=7447
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_start=7450
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_end=7614
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_start=7616
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_end=7705
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_start=7708
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_end=7900
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_start=7903
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_end=8097
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_start=8100
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_end=8292
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_start=8295
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_end=8487
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_start=8490
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_end=8682
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_start=8685
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_end=8881
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_start=8884
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_end=9078
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_start=9081
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_end=9273
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_start=9276
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_end=9492
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_start=9495
- _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_end=9729
- _globals['_SELECTDISTINCTREQUEST']._serialized_start=9748
- _globals['_SELECTDISTINCTREQUEST']._serialized_end=9925
- _globals['_DROPCOLUMNSREQUEST']._serialized_start=9928
- _globals['_DROPCOLUMNSREQUEST']._serialized_end=10102
- _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_start=10105
- _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_end=10286
- _globals['_HEADORTAILREQUEST']._serialized_start=10289
- _globals['_HEADORTAILREQUEST']._serialized_end=10462
- _globals['_HEADORTAILBYREQUEST']._serialized_start=10465
- _globals['_HEADORTAILBYREQUEST']._serialized_end=10671
- _globals['_UNGROUPREQUEST']._serialized_start=10674
- _globals['_UNGROUPREQUEST']._serialized_end=10869
- _globals['_MERGETABLESREQUEST']._serialized_start=10872
- _globals['_MERGETABLESREQUEST']._serialized_end=11045
- _globals['_SNAPSHOTTABLEREQUEST']._serialized_start=11048
- _globals['_SNAPSHOTTABLEREQUEST']._serialized_end=11202
- _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_start=11205
- _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_end=11510
- _globals['_CROSSJOINTABLESREQUEST']._serialized_start=11513
- _globals['_CROSSJOINTABLESREQUEST']._serialized_end=11808
- _globals['_NATURALJOINTABLESREQUEST']._serialized_start=11811
- _globals['_NATURALJOINTABLESREQUEST']._serialized_end=12086
- _globals['_EXACTJOINTABLESREQUEST']._serialized_start=12089
- _globals['_EXACTJOINTABLESREQUEST']._serialized_end=12362
- _globals['_LEFTJOINTABLESREQUEST']._serialized_start=12365
- _globals['_LEFTJOINTABLESREQUEST']._serialized_end=12637
- _globals['_ASOFJOINTABLESREQUEST']._serialized_start=12640
- _globals['_ASOFJOINTABLESREQUEST']._serialized_end=13105
- _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_start=13008
- _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_end=13101
- _globals['_AJRAJTABLESREQUEST']._serialized_start=13108
- _globals['_AJRAJTABLESREQUEST']._serialized_end=13402
- _globals['_MULTIJOININPUT']._serialized_start=13405
- _globals['_MULTIJOININPUT']._serialized_end=13541
- _globals['_MULTIJOINTABLESREQUEST']._serialized_start=13544
- _globals['_MULTIJOINTABLESREQUEST']._serialized_end=13708
- _globals['_RANGEJOINTABLESREQUEST']._serialized_start=13711
- _globals['_RANGEJOINTABLESREQUEST']._serialized_end=14575
- _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_start=14332
- _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_end=14450
- _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_start=14452
- _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_end=14575
- _globals['_COMBOAGGREGATEREQUEST']._serialized_start=14578
- _globals['_COMBOAGGREGATEREQUEST']._serialized_end=15216
- _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_start=14871
- _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_end=15044
- _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_start=15047
- _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_end=15212
- _globals['_AGGREGATEALLREQUEST']._serialized_start=15219
- _globals['_AGGREGATEALLREQUEST']._serialized_end=15456
- _globals['_AGGSPEC']._serialized_start=15459
- _globals['_AGGSPEC']._serialized_end=18490
- _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_start=17265
- _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_end=17357
- _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_start=17359
- _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_end=17402
- _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_start=17404
- _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_end=17444
- _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_start=17446
- _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_end=17500
- _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_start=17502
- _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_end=17549
- _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_start=17551
- _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_end=17622
- _globals['_AGGSPEC_AGGSPECSORTED']._serialized_start=17624
- _globals['_AGGSPEC_AGGSPECSORTED']._serialized_end=17720
- _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_start=17722
- _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_end=17764
- _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_start=17766
- _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_end=17824
- _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_start=17827
- _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_end=17963
- _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_start=17966
- _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_end=18275
- _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_start=18277
- _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_end=18317
- _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_start=18319
- _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_end=18334
- _globals['_AGGSPEC_AGGSPECAVG']._serialized_start=18336
- _globals['_AGGSPEC_AGGSPECAVG']._serialized_end=18348
- _globals['_AGGSPEC_AGGSPECFIRST']._serialized_start=18350
- _globals['_AGGSPEC_AGGSPECFIRST']._serialized_end=18364
- _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_start=18366
- _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_end=18381
- _globals['_AGGSPEC_AGGSPECGROUP']._serialized_start=18383
- _globals['_AGGSPEC_AGGSPECGROUP']._serialized_end=18397
- _globals['_AGGSPEC_AGGSPECLAST']._serialized_start=18399
- _globals['_AGGSPEC_AGGSPECLAST']._serialized_end=18412
- _globals['_AGGSPEC_AGGSPECMAX']._serialized_start=18414
- _globals['_AGGSPEC_AGGSPECMAX']._serialized_end=18426
- _globals['_AGGSPEC_AGGSPECMIN']._serialized_start=18428
- _globals['_AGGSPEC_AGGSPECMIN']._serialized_end=18440
- _globals['_AGGSPEC_AGGSPECSTD']._serialized_start=18442
- _globals['_AGGSPEC_AGGSPECSTD']._serialized_end=18454
- _globals['_AGGSPEC_AGGSPECSUM']._serialized_start=18456
- _globals['_AGGSPEC_AGGSPECSUM']._serialized_end=18468
- _globals['_AGGSPEC_AGGSPECVAR']._serialized_start=18470
- _globals['_AGGSPEC_AGGSPECVAR']._serialized_end=18482
- _globals['_AGGREGATEREQUEST']._serialized_start=18493
- _globals['_AGGREGATEREQUEST']._serialized_end=18841
- _globals['_AGGREGATION']._serialized_start=18844
- _globals['_AGGREGATION']._serialized_end=19898
- _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_start=19477
- _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_end=19576
- _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_start=19578
- _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_end=19617
- _globals['_AGGREGATION_AGGREGATIONCOUNTWHERE']._serialized_start=19619
- _globals['_AGGREGATION_AGGREGATIONCOUNTWHERE']._serialized_end=19680
- _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_start=19682
- _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_end=19722
- _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_start=19724
- _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_end=19801
- _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_start=19803
- _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_end=19890
- _globals['_SORTDESCRIPTOR']._serialized_start=19901
- _globals['_SORTDESCRIPTOR']._serialized_end=20126
- _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_start=20045
- _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_end=20126
- _globals['_SORTTABLEREQUEST']._serialized_start=20129
- _globals['_SORTTABLEREQUEST']._serialized_end=20345
- _globals['_FILTERTABLEREQUEST']._serialized_start=20348
- _globals['_FILTERTABLEREQUEST']._serialized_end=20563
- _globals['_SEEKROWREQUEST']._serialized_start=20566
- _globals['_SEEKROWREQUEST']._serialized_end=20815
- _globals['_SEEKROWRESPONSE']._serialized_start=20817
- _globals['_SEEKROWRESPONSE']._serialized_end=20858
- _globals['_REFERENCE']._serialized_start=20860
- _globals['_REFERENCE']._serialized_end=20892
- _globals['_LITERAL']._serialized_start=20895
- _globals['_LITERAL']._serialized_end=21040
- _globals['_VALUE']._serialized_start=21043
- _globals['_VALUE']._serialized_end=21188
- _globals['_CONDITION']._serialized_start=21191
- _globals['_CONDITION']._serialized_end=21891
- _globals['_ANDCONDITION']._serialized_start=21893
- _globals['_ANDCONDITION']._serialized_end=21970
- _globals['_ORCONDITION']._serialized_start=21972
- _globals['_ORCONDITION']._serialized_end=22048
- _globals['_NOTCONDITION']._serialized_start=22050
- _globals['_NOTCONDITION']._serialized_end=22126
- _globals['_COMPARECONDITION']._serialized_start=22129
- _globals['_COMPARECONDITION']._serialized_end=22557
- _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_start=22427
- _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_end=22557
- _globals['_INCONDITION']._serialized_start=22560
- _globals['_INCONDITION']._serialized_end=22837
- _globals['_INVOKECONDITION']._serialized_start=22840
- _globals['_INVOKECONDITION']._serialized_end=22992
- _globals['_ISNULLCONDITION']._serialized_start=22994
- _globals['_ISNULLCONDITION']._serialized_end=23076
- _globals['_MATCHESCONDITION']._serialized_start=23079
- _globals['_MATCHESCONDITION']._serialized_end=23321
- _globals['_CONTAINSCONDITION']._serialized_start=23324
- _globals['_CONTAINSCONDITION']._serialized_end=23575
- _globals['_SEARCHCONDITION']._serialized_start=23577
- _globals['_SEARCHCONDITION']._serialized_end=23692
- _globals['_FLATTENREQUEST']._serialized_start=23695
- _globals['_FLATTENREQUEST']._serialized_end=23843
- _globals['_METATABLEREQUEST']._serialized_start=23846
- _globals['_METATABLEREQUEST']._serialized_end=23996
- _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_start=23999
- _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_end=24435
- _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_start=24320
- _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_end=24435
- _globals['_CREATEINPUTTABLEREQUEST']._serialized_start=24438
- _globals['_CREATEINPUTTABLEREQUEST']._serialized_end=25174
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_start=24713
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_end=25160
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_start=25081
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_end=25101
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_start=25103
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_end=25143
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_start=25145
- _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_end=25152
- _globals['_WHEREINREQUEST']._serialized_start=25177
- _globals['_WHEREINREQUEST']._serialized_end=25436
- _globals['_COLUMNSTATISTICSREQUEST']._serialized_start=25439
- _globals['_COLUMNSTATISTICSREQUEST']._serialized_end=25673
- _globals['_SLICEREQUEST']._serialized_start=25676
- _globals['_SLICEREQUEST']._serialized_end=25897
- _globals['_BATCHTABLEREQUEST']._serialized_start=25900
- _globals['_BATCHTABLEREQUEST']._serialized_end=29319
- _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_start=25999
- _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_end=29319
- _globals['_TABLESERVICE']._serialized_start=29661
- _globals['_TABLESERVICE']._serialized_end=36118
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC']._serialized_end=10358
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_start=6972
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVESUM']._serialized_end=6995
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_start=6997
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMIN']._serialized_end=7020
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_start=7022
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEMAX']._serialized_end=7045
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_start=7047
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVEPRODUCT']._serialized_end=7074
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_start=7076
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYFILL']._serialized_end=7090
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_start=7093
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMA']._serialized_end=7255
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_start=7258
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMS']._serialized_end=7420
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_start=7423
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMIN']._serialized_end=7587
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_start=7590
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMMAX']._serialized_end=7754
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_start=7757
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYEMSTD']._serialized_end=7921
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_start=7923
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYDELTA']._serialized_end=8012
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_start=8015
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSUM']._serialized_end=8207
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_start=8210
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGGROUP']._serialized_end=8404
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_start=8407
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGAVG']._serialized_end=8599
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_start=8602
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMIN']._serialized_end=8794
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_start=8797
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGMAX']._serialized_end=8989
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_start=8992
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGPRODUCT']._serialized_end=9188
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_start=9191
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNT']._serialized_end=9385
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_start=9388
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGSTD']._serialized_end=9580
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_start=9583
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGWAVG']._serialized_end=9799
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_start=9802
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGFORMULA']._serialized_end=10036
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNTWHERE']._serialized_start=10039
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYROLLINGCOUNTWHERE']._serialized_end=10278
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVECOUNTWHERE']._serialized_start=10280
+ _globals['_UPDATEBYREQUEST_UPDATEBYOPERATION_UPDATEBYCOLUMN_UPDATEBYSPEC_UPDATEBYCUMULATIVECOUNTWHERE']._serialized_end=10350
+ _globals['_SELECTDISTINCTREQUEST']._serialized_start=10369
+ _globals['_SELECTDISTINCTREQUEST']._serialized_end=10546
+ _globals['_DROPCOLUMNSREQUEST']._serialized_start=10549
+ _globals['_DROPCOLUMNSREQUEST']._serialized_end=10723
+ _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_start=10726
+ _globals['_UNSTRUCTUREDFILTERTABLEREQUEST']._serialized_end=10907
+ _globals['_HEADORTAILREQUEST']._serialized_start=10910
+ _globals['_HEADORTAILREQUEST']._serialized_end=11083
+ _globals['_HEADORTAILBYREQUEST']._serialized_start=11086
+ _globals['_HEADORTAILBYREQUEST']._serialized_end=11292
+ _globals['_UNGROUPREQUEST']._serialized_start=11295
+ _globals['_UNGROUPREQUEST']._serialized_end=11490
+ _globals['_MERGETABLESREQUEST']._serialized_start=11493
+ _globals['_MERGETABLESREQUEST']._serialized_end=11666
+ _globals['_SNAPSHOTTABLEREQUEST']._serialized_start=11669
+ _globals['_SNAPSHOTTABLEREQUEST']._serialized_end=11823
+ _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_start=11826
+ _globals['_SNAPSHOTWHENTABLEREQUEST']._serialized_end=12131
+ _globals['_CROSSJOINTABLESREQUEST']._serialized_start=12134
+ _globals['_CROSSJOINTABLESREQUEST']._serialized_end=12429
+ _globals['_NATURALJOINTABLESREQUEST']._serialized_start=12432
+ _globals['_NATURALJOINTABLESREQUEST']._serialized_end=12707
+ _globals['_EXACTJOINTABLESREQUEST']._serialized_start=12710
+ _globals['_EXACTJOINTABLESREQUEST']._serialized_end=12983
+ _globals['_LEFTJOINTABLESREQUEST']._serialized_start=12986
+ _globals['_LEFTJOINTABLESREQUEST']._serialized_end=13258
+ _globals['_ASOFJOINTABLESREQUEST']._serialized_start=13261
+ _globals['_ASOFJOINTABLESREQUEST']._serialized_end=13726
+ _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_start=13629
+ _globals['_ASOFJOINTABLESREQUEST_MATCHRULE']._serialized_end=13722
+ _globals['_AJRAJTABLESREQUEST']._serialized_start=13729
+ _globals['_AJRAJTABLESREQUEST']._serialized_end=14023
+ _globals['_MULTIJOININPUT']._serialized_start=14026
+ _globals['_MULTIJOININPUT']._serialized_end=14162
+ _globals['_MULTIJOINTABLESREQUEST']._serialized_start=14165
+ _globals['_MULTIJOINTABLESREQUEST']._serialized_end=14329
+ _globals['_RANGEJOINTABLESREQUEST']._serialized_start=14332
+ _globals['_RANGEJOINTABLESREQUEST']._serialized_end=15196
+ _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_start=14953
+ _globals['_RANGEJOINTABLESREQUEST_RANGESTARTRULE']._serialized_end=15071
+ _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_start=15073
+ _globals['_RANGEJOINTABLESREQUEST_RANGEENDRULE']._serialized_end=15196
+ _globals['_COMBOAGGREGATEREQUEST']._serialized_start=15199
+ _globals['_COMBOAGGREGATEREQUEST']._serialized_end=15837
+ _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_start=15492
+ _globals['_COMBOAGGREGATEREQUEST_AGGREGATE']._serialized_end=15665
+ _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_start=15668
+ _globals['_COMBOAGGREGATEREQUEST_AGGTYPE']._serialized_end=15833
+ _globals['_AGGREGATEALLREQUEST']._serialized_start=15840
+ _globals['_AGGREGATEALLREQUEST']._serialized_end=16077
+ _globals['_AGGSPEC']._serialized_start=16080
+ _globals['_AGGSPEC']._serialized_end=19111
+ _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_start=17886
+ _globals['_AGGSPEC_AGGSPECAPPROXIMATEPERCENTILE']._serialized_end=17978
+ _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_start=17980
+ _globals['_AGGSPEC_AGGSPECCOUNTDISTINCT']._serialized_end=18023
+ _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_start=18025
+ _globals['_AGGSPEC_AGGSPECDISTINCT']._serialized_end=18065
+ _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_start=18067
+ _globals['_AGGSPEC_AGGSPECFORMULA']._serialized_end=18121
+ _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_start=18123
+ _globals['_AGGSPEC_AGGSPECMEDIAN']._serialized_end=18170
+ _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_start=18172
+ _globals['_AGGSPEC_AGGSPECPERCENTILE']._serialized_end=18243
+ _globals['_AGGSPEC_AGGSPECSORTED']._serialized_start=18245
+ _globals['_AGGSPEC_AGGSPECSORTED']._serialized_end=18341
+ _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_start=18343
+ _globals['_AGGSPEC_AGGSPECSORTEDCOLUMN']._serialized_end=18385
+ _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_start=18387
+ _globals['_AGGSPEC_AGGSPECTDIGEST']._serialized_end=18445
+ _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_start=18448
+ _globals['_AGGSPEC_AGGSPECUNIQUE']._serialized_end=18584
+ _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_start=18587
+ _globals['_AGGSPEC_AGGSPECNONUNIQUESENTINEL']._serialized_end=18896
+ _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_start=18898
+ _globals['_AGGSPEC_AGGSPECWEIGHTED']._serialized_end=18938
+ _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_start=18940
+ _globals['_AGGSPEC_AGGSPECABSSUM']._serialized_end=18955
+ _globals['_AGGSPEC_AGGSPECAVG']._serialized_start=18957
+ _globals['_AGGSPEC_AGGSPECAVG']._serialized_end=18969
+ _globals['_AGGSPEC_AGGSPECFIRST']._serialized_start=18971
+ _globals['_AGGSPEC_AGGSPECFIRST']._serialized_end=18985
+ _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_start=18987
+ _globals['_AGGSPEC_AGGSPECFREEZE']._serialized_end=19002
+ _globals['_AGGSPEC_AGGSPECGROUP']._serialized_start=19004
+ _globals['_AGGSPEC_AGGSPECGROUP']._serialized_end=19018
+ _globals['_AGGSPEC_AGGSPECLAST']._serialized_start=19020
+ _globals['_AGGSPEC_AGGSPECLAST']._serialized_end=19033
+ _globals['_AGGSPEC_AGGSPECMAX']._serialized_start=19035
+ _globals['_AGGSPEC_AGGSPECMAX']._serialized_end=19047
+ _globals['_AGGSPEC_AGGSPECMIN']._serialized_start=19049
+ _globals['_AGGSPEC_AGGSPECMIN']._serialized_end=19061
+ _globals['_AGGSPEC_AGGSPECSTD']._serialized_start=19063
+ _globals['_AGGSPEC_AGGSPECSTD']._serialized_end=19075
+ _globals['_AGGSPEC_AGGSPECSUM']._serialized_start=19077
+ _globals['_AGGSPEC_AGGSPECSUM']._serialized_end=19089
+ _globals['_AGGSPEC_AGGSPECVAR']._serialized_start=19091
+ _globals['_AGGSPEC_AGGSPECVAR']._serialized_end=19103
+ _globals['_AGGREGATEREQUEST']._serialized_start=19114
+ _globals['_AGGREGATEREQUEST']._serialized_end=19462
+ _globals['_AGGREGATION']._serialized_start=19465
+ _globals['_AGGREGATION']._serialized_end=20519
+ _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_start=20098
+ _globals['_AGGREGATION_AGGREGATIONCOLUMNS']._serialized_end=20197
+ _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_start=20199
+ _globals['_AGGREGATION_AGGREGATIONCOUNT']._serialized_end=20238
+ _globals['_AGGREGATION_AGGREGATIONCOUNTWHERE']._serialized_start=20240
+ _globals['_AGGREGATION_AGGREGATIONCOUNTWHERE']._serialized_end=20301
+ _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_start=20303
+ _globals['_AGGREGATION_AGGREGATIONROWKEY']._serialized_end=20343
+ _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_start=20345
+ _globals['_AGGREGATION_AGGREGATIONPARTITION']._serialized_end=20422
+ _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_start=20424
+ _globals['_AGGREGATION_AGGREGATIONFORMULA']._serialized_end=20511
+ _globals['_SORTDESCRIPTOR']._serialized_start=20522
+ _globals['_SORTDESCRIPTOR']._serialized_end=20747
+ _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_start=20666
+ _globals['_SORTDESCRIPTOR_SORTDIRECTION']._serialized_end=20747
+ _globals['_SORTTABLEREQUEST']._serialized_start=20750
+ _globals['_SORTTABLEREQUEST']._serialized_end=20966
+ _globals['_FILTERTABLEREQUEST']._serialized_start=20969
+ _globals['_FILTERTABLEREQUEST']._serialized_end=21184
+ _globals['_SEEKROWREQUEST']._serialized_start=21187
+ _globals['_SEEKROWREQUEST']._serialized_end=21436
+ _globals['_SEEKROWRESPONSE']._serialized_start=21438
+ _globals['_SEEKROWRESPONSE']._serialized_end=21479
+ _globals['_REFERENCE']._serialized_start=21481
+ _globals['_REFERENCE']._serialized_end=21513
+ _globals['_LITERAL']._serialized_start=21516
+ _globals['_LITERAL']._serialized_end=21661
+ _globals['_VALUE']._serialized_start=21664
+ _globals['_VALUE']._serialized_end=21809
+ _globals['_CONDITION']._serialized_start=21812
+ _globals['_CONDITION']._serialized_end=22512
+ _globals['_ANDCONDITION']._serialized_start=22514
+ _globals['_ANDCONDITION']._serialized_end=22591
+ _globals['_ORCONDITION']._serialized_start=22593
+ _globals['_ORCONDITION']._serialized_end=22669
+ _globals['_NOTCONDITION']._serialized_start=22671
+ _globals['_NOTCONDITION']._serialized_end=22747
+ _globals['_COMPARECONDITION']._serialized_start=22750
+ _globals['_COMPARECONDITION']._serialized_end=23178
+ _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_start=23048
+ _globals['_COMPARECONDITION_COMPAREOPERATION']._serialized_end=23178
+ _globals['_INCONDITION']._serialized_start=23181
+ _globals['_INCONDITION']._serialized_end=23458
+ _globals['_INVOKECONDITION']._serialized_start=23461
+ _globals['_INVOKECONDITION']._serialized_end=23613
+ _globals['_ISNULLCONDITION']._serialized_start=23615
+ _globals['_ISNULLCONDITION']._serialized_end=23697
+ _globals['_MATCHESCONDITION']._serialized_start=23700
+ _globals['_MATCHESCONDITION']._serialized_end=23942
+ _globals['_CONTAINSCONDITION']._serialized_start=23945
+ _globals['_CONTAINSCONDITION']._serialized_end=24196
+ _globals['_SEARCHCONDITION']._serialized_start=24198
+ _globals['_SEARCHCONDITION']._serialized_end=24313
+ _globals['_FLATTENREQUEST']._serialized_start=24316
+ _globals['_FLATTENREQUEST']._serialized_end=24464
+ _globals['_METATABLEREQUEST']._serialized_start=24467
+ _globals['_METATABLEREQUEST']._serialized_end=24617
+ _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_start=24620
+ _globals['_RUNCHARTDOWNSAMPLEREQUEST']._serialized_end=25056
+ _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_start=24941
+ _globals['_RUNCHARTDOWNSAMPLEREQUEST_ZOOMRANGE']._serialized_end=25056
+ _globals['_CREATEINPUTTABLEREQUEST']._serialized_start=25059
+ _globals['_CREATEINPUTTABLEREQUEST']._serialized_end=25795
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_start=25334
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND']._serialized_end=25781
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_start=25702
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYAPPENDONLY']._serialized_end=25722
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_start=25724
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_INMEMORYKEYBACKED']._serialized_end=25764
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_start=25766
+ _globals['_CREATEINPUTTABLEREQUEST_INPUTTABLEKIND_BLINK']._serialized_end=25773
+ _globals['_WHEREINREQUEST']._serialized_start=25798
+ _globals['_WHEREINREQUEST']._serialized_end=26057
+ _globals['_COLUMNSTATISTICSREQUEST']._serialized_start=26060
+ _globals['_COLUMNSTATISTICSREQUEST']._serialized_end=26294
+ _globals['_SLICEREQUEST']._serialized_start=26297
+ _globals['_SLICEREQUEST']._serialized_end=26518
+ _globals['_BATCHTABLEREQUEST']._serialized_start=26521
+ _globals['_BATCHTABLEREQUEST']._serialized_end=29940
+ _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_start=26620
+ _globals['_BATCHTABLEREQUEST_OPERATION']._serialized_end=29940
+ _globals['_TABLESERVICE']._serialized_start=30282
+ _globals['_TABLESERVICE']._serialized_end=36739
# @@protoc_insertion_point(module_scope)
diff --git a/py/client/pydeephaven/table.py b/py/client/pydeephaven/table.py
index fb8d221fac5..6ab289fef1d 100644
--- a/py/client/pydeephaven/table.py
+++ b/py/client/pydeephaven/table.py
@@ -665,7 +665,8 @@ def agg_all_by(self, agg: Aggregation, by: Union[str, List[str]]) -> Table:
"""
return super(Table, self).agg_all_by(agg, by)
- def update_by(self, ops: Union[UpdateByOperation, List[UpdateByOperation]], by: Union[str, List[str]]) -> Table:
+ def update_by(self, ops: Union[UpdateByOperation, List[UpdateByOperation]],
+ by: Union[str, List[str]] = None) -> Table:
"""The update_by method creates a table with additional columns calculated from
window-based aggregations of columns in this table. The aggregations are defined by the provided operations,
which support incremental aggregations over the corresponding rows in the table. The aggregations will
@@ -674,7 +675,8 @@ def update_by(self, ops: Union[UpdateByOperation, List[UpdateByOperation]], by:
Args:
ops (Union[UpdateByOperatoin, List[UpdateByOperation]]): the UpdateByOperation(s) to be applied
- by (Union[str, List[str]]): the group-by column name(s)
+ by (Union[str, List[str]]): the group-by column name(s), defaults to None, meaning all calculations are
+ performed over the entire table
Returns:
a Table object
diff --git a/py/client/pydeephaven/updateby.py b/py/client/pydeephaven/updateby.py
index 3b65cbb94eb..77f40e5c9d6 100644
--- a/py/client/pydeephaven/updateby.py
+++ b/py/client/pydeephaven/updateby.py
@@ -31,6 +31,7 @@
_GrpcUpdateByRollingStd = _GrpcUpdateBySpec.UpdateByRollingStd
_GrpcUpdateByRollingWAvg = _GrpcUpdateBySpec.UpdateByRollingWAvg
_GrpcUpdateByRollingFormula = _GrpcUpdateBySpec.UpdateByRollingFormula
+_GrpcUpdateByRollingCountWhere = _GrpcUpdateBySpec.UpdateByRollingCountWhere
_GrpcUpdateByDeltaOptions = table_pb2.UpdateByDeltaOptions
_GrpcUpdateByWindowScale = table_pb2.UpdateByWindowScale
_GrpcUpdateByWindowTicks = _GrpcUpdateByWindowScale.UpdateByWindowTicks
@@ -185,6 +186,27 @@ def cum_max(cols: Union[str, List[str]]) -> UpdateByOperation:
return UpdateByOperation(ub_column=ub_column)
+def cum_count_where(col: str, filters: Union[str, List[str]]) -> UpdateByOperation:
+ """Creates a cumulative count where UpdateByOperation that counts the number of values that pass the provided
+ filters.
+
+ Args:
+ col (str): the column to hold the counts of rows that pass the filter condition columns.
+ filters (Union[str, Filter, Sequence[str], Sequence[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
+
+ Returns:
+ an UpdateByOperation
+ """
+ if not isinstance(col, str):
+ raise DHError(message="count_where aggregation requires a string value for the 'col' argument.")
+ filters = to_list(filters)
+
+ ub_spec = _GrpcUpdateBySpec(count_where=_GrpcUpdateBySpec.UpdateByCumulativeCountWhere(result_column=col, filters=filters))
+ ub_column = _GrpcUpdateByColumn(spec=ub_spec)
+ return UpdateByOperation(ub_column=ub_column)
+
+
def forward_fill(cols: Union[str, List[str]]) -> UpdateByOperation:
"""Creates a forward fill UpdateByOperation for the supplied column names. Null values in the column(s) are
replaced by the last known non-null values. This operation is forward only.
@@ -1601,3 +1623,117 @@ def rolling_formula_time(ts_col: str, formula: str, formula_param: str, cols: Un
return UpdateByOperation(ub_column=ub_column)
except Exception as e:
raise DHError(e, "failed to create a rolling formula (time) UpdateByOperation.") from e
+
+
+def rolling_count_where_tick(col: str, filters: Union[str, List[str]],
+ rev_ticks: int, fwd_ticks: int = 0) -> UpdateByOperation:
+ """Creates a rolling count where UpdateByOperation that counts the number of values that pass the provided
+ filters, using ticks as the windowing unit. Ticks are row counts, and you may specify the reverse and forward
+ window in number of rows to include. The current row is considered to belong to the reverse window but not the
+ forward window. Also, negative values are allowed and can be used to generate completely forward or completely
+ reverse windows.
+
+ Here are some examples of window values:
+ | `rev_ticks = 1, fwd_ticks = 0` - contains only the current row
+ | `rev_ticks = 10, fwd_ticks = 0` - contains 9 previous rows and the current row
+ | `rev_ticks = 0, fwd_ticks = 10` - contains the following 10 rows, excludes the current row
+ | `rev_ticks = 10, fwd_ticks = 10` - contains the previous 9 rows, the current row and the 10 rows following
+ | `rev_ticks = 10, fwd_ticks = -5` - contains 5 rows, beginning at 9 rows before, ending at 5 rows before the
+ current row (inclusive)
+ | `rev_ticks = 11, fwd_ticks = -1` - contains 10 rows, beginning at 10 rows before, ending at 1 row before the
+ current row (inclusive)
+ | `rev_ticks = -5, fwd_ticks = 10` - contains 5 rows, beginning 5 rows following, ending at 10 rows following the
+ current row (inclusive)
+
+ Args:
+ col (str): the column to hold the counts of rows that pass the filter condition columns.
+ filters (Union[str, Filter, Sequence[str], Sequence[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
+ rev_ticks (int): the look-behind window size (in rows/ticks)
+ fwd_ticks (int): the look-forward window size (int rows/ticks), default is 0
+
+ Returns:
+ an UpdateByOperation
+
+ Raises:
+ DHError
+ """
+ if not isinstance(col, str):
+ raise DHError(message="count_where aggregation requires a string value for the 'col' argument.")
+ filters = to_list(filters)
+
+ try:
+ rev_window_scale = _GrpcUpdateByWindowScale(ticks=_GrpcUpdateByWindowTicks(ticks=rev_ticks))
+ fwd_window_scale = _GrpcUpdateByWindowScale(ticks=_GrpcUpdateByWindowTicks(ticks=fwd_ticks))
+ ub_count_where = _GrpcUpdateByRollingCountWhere(reverse_window_scale=rev_window_scale,
+ forward_window_scale=fwd_window_scale,
+ result_column=col,
+ filters=filters)
+ ub_spec = _GrpcUpdateBySpec(rolling_count_where=ub_count_where)
+ ub_column = _GrpcUpdateByColumn(spec=ub_spec)
+ return UpdateByOperation(ub_column=ub_column)
+ except Exception as e:
+ raise DHError(e, "failed to create a rolling count_where UpdateByOperation.") from e
+
+
+def rolling_count_where_time(ts_col: str, col: str, filters: Union[str, List[str]],
+ rev_time: Union[int, str], fwd_time: Union[int, str] = 0) -> UpdateByOperation:
+ """Creates a rolling count where UpdateByOperation that counts the number of values that pass the provided
+ filters, using time as the windowing unit. This function accepts nanoseconds or time strings as the reverse and
+ forward window parameters. Negative values are allowed and can be used to generate completely forward or completely
+ reverse windows. A row containing a null in the timestamp column belongs to no window and will not be considered in
+ the windows of other rows; its output will be null.
+
+ Here are some examples of window values:
+ | `rev_time = 0, fwd_time = 0` - contains rows that exactly match the current row timestamp
+ | `rev_time = "PT00:10:00", fwd_time = "0"` - contains rows from 10m before through the current row timestamp (
+ inclusive)
+ | `rev_time = 0, fwd_time = 600_000_000_000` - contains rows from the current row through 10m following the
+ current row timestamp (inclusive)
+ | `rev_time = "PT00:10:00", fwd_time = "PT00:10:00"` - contains rows from 10m before through 10m following
+ the current row timestamp (inclusive)
+ | `rev_time = "PT00:10:00", fwd_time = "-PT00:05:00"` - contains rows from 10m before through 5m before the
+ current row timestamp (inclusive), this is a purely backwards looking window
+ | `rev_time = "-PT00:05:00", fwd_time = "PT00:10:00"` - contains rows from 5m following through 10m
+ following the current row timestamp (inclusive), this is a purely forwards looking window
+
+ Args:
+ ts_col (str): the timestamp column for determining the window
+ col (str): the column to hold the counts of rows that pass the filter condition columns.
+ filters (Union[str, Filter, Sequence[str], Sequence[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
+ rev_time (int): the look-behind window size, can be expressed as an integer in nanoseconds or a time
+ interval string, e.g. "PT00:00:00.001" or "PT5M"
+ fwd_time (int): the look-ahead window size, can be expressed as an integer in nanoseconds or a time
+ interval string, e.g. "PT00:00:00.001" or "PT5M", default is 0
+
+ Returns:
+ an UpdateByOperation
+
+ Raises:
+ DHError
+ """
+ if not isinstance(col, str):
+ raise DHError(message="count_where aggregation requires a string value for the 'col' argument.")
+ filters = to_list(filters)
+
+ try:
+ if isinstance(rev_time, str):
+ rev_window_scale = _GrpcUpdateByWindowScale(time=_GrpcUpdateByWindowTime(column=ts_col, duration_string=rev_time))
+ else:
+ rev_window_scale = _GrpcUpdateByWindowScale(time=_GrpcUpdateByWindowTime(column=ts_col, nanos=rev_time))
+
+ if isinstance(fwd_time, str):
+ fwd_window_scale = _GrpcUpdateByWindowScale(time=_GrpcUpdateByWindowTime(column=ts_col, duration_string=fwd_time))
+ else:
+ fwd_window_scale = _GrpcUpdateByWindowScale(time=_GrpcUpdateByWindowTime(column=ts_col, nanos=fwd_time))
+
+ ub_count_where = _GrpcUpdateByRollingCountWhere(reverse_window_scale=rev_window_scale,
+ forward_window_scale=fwd_window_scale,
+ result_column=col,
+ filters=filters)
+ ub_spec = _GrpcUpdateBySpec(rolling_count_where=ub_count_where)
+ ub_column = _GrpcUpdateByColumn(spec=ub_spec)
+ return UpdateByOperation(ub_column=ub_column)
+ except Exception as e:
+ raise DHError(e, "failed to create a rolling count_where UpdateByOperation.") from e
diff --git a/py/client/tests/test_updateby.py b/py/client/tests/test_updateby.py
index d8c11f5c54a..3a196b31fa3 100644
--- a/py/client/tests/test_updateby.py
+++ b/py/client/tests/test_updateby.py
@@ -11,7 +11,8 @@
cum_sum, cum_prod, cum_min, cum_max, forward_fill, delta, rolling_sum_tick, rolling_sum_time, \
rolling_group_tick, rolling_group_time, rolling_avg_tick, rolling_avg_time, rolling_min_tick, rolling_min_time, \
rolling_max_tick, rolling_max_time, rolling_prod_tick, rolling_prod_time, rolling_count_tick, rolling_count_time, \
- rolling_std_tick, rolling_std_time, rolling_wavg_tick, rolling_wavg_time, rolling_formula_tick, rolling_formula_time
+ rolling_std_tick, rolling_std_time, rolling_wavg_tick, rolling_wavg_time, rolling_formula_tick, rolling_formula_time, \
+ cum_count_where, rolling_count_where_tick, rolling_count_where_time
from tests.testbase import BaseTestCase
@@ -89,6 +90,12 @@ def setUpClass(cls) -> None:
delta(cols=simple_op_pairs, delta_control=DeltaControl.ZERO_DOMINATES),
]
+ cls.simple_ops_one_output = [
+ cum_count_where(col='count_1', filters='a > 5'),
+ cum_count_where(col='count_2', filters='a > 0 && a < 5'),
+ cum_count_where(col='count_3', filters=['a > 0', 'a < 5']),
+ ]
+
# Rolling Operators list shared with test_rolling_ops / test_rolling_ops_proxy
cls.rolling_ops = [
# rolling sum
@@ -182,6 +189,14 @@ def test_simple_ops(self):
self.assertEqual(len(rt.schema), 2 + len(t.schema))
self.assertGreaterEqual(rt.size, t.size)
+ def test_simple_ops_one_output(self):
+ for op in self.simple_ops:
+ with self.subTest(op):
+ for t in (self.static_table, self.ticking_table):
+ rt = t.update_by(ops=op, by="e")
+ self.assertTrue(rt.is_refreshing is t.is_refreshing)
+ self.assertEqual(len(rt.schema), 2 + len(t.schema))
+ self.assertGreaterEqual(rt.size, t.size)
def test_em(self):
for op in self.em_ops:
@@ -217,6 +232,55 @@ def test_multiple_ops(self):
if not rt.is_refreshing:
self.assertEqual(rt.size, t.size)
+ def test_cum_count_where_output(self):
+ """
+ Test and validation of the cum_count_where feature
+ """
+ test_table = self.session.empty_table(4).update(["a=ii", "b=ii%2"])
+ count_aggs = [
+ cum_count_where(col="count1", filters="a >= 1"),
+ cum_count_where(col="count2", filters="a >= 1 && b == 0"),
+ ]
+ result_table = test_table.update_by(ops=count_aggs)
+ self.assertEqual(result_table.size, 4)
+
+ # get the table as a local pandas dataframe
+ df = result_table.to_arrow().to_pandas()
+ # assert the values meet expectations
+ self.assertEqual(df.loc[0, "count1"], 0)
+ self.assertEqual(df.loc[1, "count1"], 1)
+ self.assertEqual(df.loc[2, "count1"], 2)
+ self.assertEqual(df.loc[3, "count1"], 3)
+
+ self.assertEqual(df.loc[0, "count2"], 0)
+ self.assertEqual(df.loc[1, "count2"], 0)
+ self.assertEqual(df.loc[2, "count2"], 1)
+ self.assertEqual(df.loc[3, "count2"], 1)
+
+ def test_rolling_count_where_output(self):
+ """
+ Test and validation of the rolling_count_where feature
+ """
+ test_table = self.session.empty_table(4).update(["a=ii", "b=ii%2"])
+ count_aggs = [
+ rolling_count_where_tick(col="count1", filters="a >= 1", rev_ticks=2),
+ rolling_count_where_tick(col="count2", filters="a >= 1 && b == 0", rev_ticks=2),
+ ]
+ result_table = test_table.update_by(ops=count_aggs)
+ self.assertEqual(result_table.size, 4)
+
+ # get the table as a local pandas dataframe
+ df = result_table.to_arrow().to_pandas()
+ # assert the values meet expectations
+ self.assertEqual(df.loc[0, "count1"], 0)
+ self.assertEqual(df.loc[1, "count1"], 1)
+ self.assertEqual(df.loc[2, "count1"], 2)
+ self.assertEqual(df.loc[3, "count1"], 2)
+
+ self.assertEqual(df.loc[0, "count2"], 0)
+ self.assertEqual(df.loc[1, "count2"], 0)
+ self.assertEqual(df.loc[2, "count2"], 1)
+ self.assertEqual(df.loc[3, "count2"], 1)
if __name__ == '__main__':
unittest.main()
diff --git a/py/server/deephaven/agg.py b/py/server/deephaven/agg.py
index 45f7c4668ab..8de070d32af 100644
--- a/py/server/deephaven/agg.py
+++ b/py/server/deephaven/agg.py
@@ -119,7 +119,9 @@ def count_where(col: str, filters: Union[str, Filter, Sequence[str], Sequence[Fi
filters.
Args:
- col (str): the column to hold the counts of each distinct group
+ col (str): the column to hold the counts of rows that pass the filter condition
+ filters (Union[str, Filter, Sequence[str], Sequence[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
Returns:
an aggregation
diff --git a/py/server/deephaven/updateby.py b/py/server/deephaven/updateby.py
index 8c68f5ffbe7..284f59f3490 100644
--- a/py/server/deephaven/updateby.py
+++ b/py/server/deephaven/updateby.py
@@ -10,6 +10,7 @@
from deephaven import DHError
from deephaven._wrapper import JObjectWrapper
from deephaven.jcompat import to_sequence
+from deephaven.filters import Filter, and_
_JUpdateByOperation = jpy.get_type("io.deephaven.api.updateby.UpdateByOperation")
_JBadDataBehavior = jpy.get_type("io.deephaven.api.updateby.BadDataBehavior")
@@ -563,6 +564,31 @@ def cum_max(cols: Union[str, List[str]]) -> UpdateByOperation:
raise DHError(e, "failed to create a cumulative maximum UpdateByOperation.") from e
+def cum_count_where(col: str, filters: Union[str, Filter, List[str], List[Filter]]) -> UpdateByOperation:
+ """Creates a cumulative count where UpdateByOperation that counts the number of values that pass the provided
+ filters.
+
+ Args:
+ col (str): the column to hold the counts of rows that pass the filter condition columns.
+ filters (Union[str, Filter, List[str], List[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
+
+ Returns:
+ an UpdateByOperation
+
+ Raises:
+ DHError
+ """
+ if not isinstance(col, str):
+ raise DHError(message="count_where aggregation requires a string value for the 'col' argument.")
+ filters = to_sequence(filters)
+
+ try:
+ return UpdateByOperation(j_updateby_op=_JUpdateByOperation.CumCountWhere(col, and_(filters).j_filter))
+ except Exception as e:
+ raise DHError(e, "failed to create a cumulative count_where UpdateByOperation.") from e
+
+
def forward_fill(cols: Union[str, List[str]]) -> UpdateByOperation:
"""Creates a forward fill UpdateByOperation for the supplied column names. Null values in the columns are
replaced by the last known non-null values. This operation is forward only.
@@ -1494,4 +1520,96 @@ def rolling_formula_time(ts_col: str, formula: str, formula_param: str = None, c
cols = to_sequence(cols)
return UpdateByOperation(j_updateby_op=_JUpdateByOperation.RollingFormula(ts_col, rev_time, fwd_time, formula, formula_param, *cols))
except Exception as e:
- raise DHError(e, "failed to create a rolling formula (time) UpdateByOperation.") from e
\ No newline at end of file
+ raise DHError(e, "failed to create a rolling formula (time) UpdateByOperation.") from e
+
+
+def rolling_count_where_tick(col: str, filters: Union[str, Filter, List[str], List[Filter]],
+ rev_ticks: int, fwd_ticks: int = 0) -> UpdateByOperation:
+ """Creates a rolling count where UpdateByOperation that counts the number of values that pass the provided
+ filters, using ticks as the windowing unit. Ticks are row counts, and you may specify the reverse and forward
+ window in number of rows to include. The current row is considered to belong to the reverse window but not the
+ forward window. Also, negative values are allowed and can be used to generate completely forward or completely
+ reverse windows.
+
+ Here are some examples of window values:
+ | `rev_ticks = 1, fwd_ticks = 0` - contains only the current row
+ | `rev_ticks = 10, fwd_ticks = 0` - contains 9 previous rows and the current row
+ | `rev_ticks = 0, fwd_ticks = 10` - contains the following 10 rows, excludes the current row
+ | `rev_ticks = 10, fwd_ticks = 10` - contains the previous 9 rows, the current row and the 10 rows following
+ | `rev_ticks = 10, fwd_ticks = -5` - contains 5 rows, beginning at 9 rows before, ending at 5 rows before the
+ current row (inclusive)
+ | `rev_ticks = 11, fwd_ticks = -1` - contains 10 rows, beginning at 10 rows before, ending at 1 row before the
+ current row (inclusive)
+ | `rev_ticks = -5, fwd_ticks = 10` - contains 5 rows, beginning 5 rows following, ending at 10 rows following the
+ current row (inclusive)
+
+ Args:
+ col (str): the column to hold the counts of rows that pass the filter condition columns.
+ filters (Union[str, Filter, List[str], List[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
+ rev_ticks (int): the look-behind window size (in rows/ticks)
+ fwd_ticks (int): the look-forward window size (int rows/ticks), default is 0
+
+ Returns:
+ an UpdateByOperation
+
+ Raises:
+ DHError
+ """
+ if not isinstance(col, str):
+ raise DHError(message="count_where aggregation requires a string value for the 'col' argument.")
+ filters = to_sequence(filters)
+
+ try:
+ return UpdateByOperation(j_updateby_op=_JUpdateByOperation.RollingCountWhere(rev_ticks, fwd_ticks, col, and_(filters).j_filter))
+ except Exception as e:
+ raise DHError(e, "failed to create a rolling count_where UpdateByOperation.") from e
+
+
+def rolling_count_where_time(ts_col: str, col: str, filters: Union[str, Filter, List[str], List[Filter]],
+ rev_time: Union[int, str], fwd_time: Union[int, str] = 0) -> UpdateByOperation:
+ """Creates a rolling count where UpdateByOperation that counts the number of values that pass the provided
+ filters, using time as the windowing unit. This function accepts nanoseconds or time strings as the reverse and
+ forward window parameters. Negative values are allowed and can be used to generate completely forward or completely
+ reverse windows. A row containing a null in the timestamp column belongs to no window and will not be considered in
+ the windows of other rows; its output will be null.
+
+ Here are some examples of window values:
+ | `rev_time = 0, fwd_time = 0` - contains rows that exactly match the current row timestamp
+ | `rev_time = "PT00:10:00", fwd_time = "0"` - contains rows from 10m before through the current row timestamp (
+ inclusive)
+ | `rev_time = 0, fwd_time = 600_000_000_000` - contains rows from the current row through 10m following the
+ current row timestamp (inclusive)
+ | `rev_time = "PT00:10:00", fwd_time = "PT00:10:00"` - contains rows from 10m before through 10m following
+ the current row timestamp (inclusive)
+ | `rev_time = "PT00:10:00", fwd_time = "-PT00:05:00"` - contains rows from 10m before through 5m before the
+ current row timestamp (inclusive), this is a purely backwards looking window
+ | `rev_time = "-PT00:05:00", fwd_time = "PT00:10:00"` - contains rows from 5m following through 10m
+ following the current row timestamp (inclusive), this is a purely forwards looking window
+
+ Args:
+ ts_col (str): the timestamp column for determining the window
+ col (str): the column to hold the counts of rows that pass the filter condition columns.
+ filters (Union[str, Filter, List[str], List[Filter]], optional): the filter condition
+ expression(s) or Filter object(s)
+ rev_time (int): the look-behind window size, can be expressed as an integer in nanoseconds or a time
+ interval string, e.g. "PT00:00:00.001" or "PT5M"
+ fwd_time (int): the look-ahead window size, can be expressed as an integer in nanoseconds or a time
+ interval string, e.g. "PT00:00:00.001" or "PT5M", default is 0
+
+ Returns:
+ an UpdateByOperation
+
+ Raises:
+ DHError
+ """
+ if not isinstance(col, str):
+ raise DHError(message="count_where aggregation requires a string value for the 'col' argument.")
+ filters = to_sequence(filters)
+
+ try:
+ rev_time = _JDateTimeUtils.parseDurationNanos(rev_time) if isinstance(rev_time, str) else rev_time
+ fwd_time = _JDateTimeUtils.parseDurationNanos(fwd_time) if isinstance(fwd_time, str) else fwd_time
+ return UpdateByOperation(j_updateby_op=_JUpdateByOperation.RollingCountWhere(ts_col, rev_time, fwd_time, col, and_(filters).j_filter))
+ except Exception as e:
+ raise DHError(e, "failed to create a rolling count_where UpdateByOperation.") from e
diff --git a/py/server/tests/test_updateby.py b/py/server/tests/test_updateby.py
index 150d41d51e5..9fe16a422cc 100644
--- a/py/server/tests/test_updateby.py
+++ b/py/server/tests/test_updateby.py
@@ -4,13 +4,15 @@
import unittest
-from deephaven import read_csv, time_table, update_graph
+from deephaven import read_csv, time_table, update_graph, empty_table
from deephaven.updateby import BadDataBehavior, MathContext, OperationControl, DeltaControl, ema_tick, ema_time, \
ems_tick, ems_time, emmin_tick, emmin_time, emmax_tick, emmax_time, emstd_tick, emstd_time,\
cum_sum, cum_prod, cum_min, cum_max, forward_fill, delta, rolling_sum_tick, rolling_sum_time, \
rolling_group_tick, rolling_group_time, rolling_avg_tick, rolling_avg_time, rolling_min_tick, rolling_min_time, \
rolling_max_tick, rolling_max_time, rolling_prod_tick, rolling_prod_time, rolling_count_tick, rolling_count_time, \
- rolling_std_tick, rolling_std_time, rolling_wavg_tick, rolling_wavg_time, rolling_formula_tick, rolling_formula_time
+ rolling_std_tick, rolling_std_time, rolling_wavg_tick, rolling_wavg_time, rolling_formula_tick, rolling_formula_time, \
+ cum_count_where, rolling_count_where_tick, rolling_count_where_time
+from deephaven.pandas import to_pandas
from tests.testbase import BaseTestCase
from deephaven.execution_context import get_exec_ctx, make_user_exec_ctx
@@ -81,6 +83,12 @@ def setUpClass(cls) -> None:
delta(cols=simple_op_pairs, delta_control=DeltaControl.ZERO_DOMINATES),
]
+ cls.simple_ops_one_output = [
+ cum_count_where(col='count_1', filters='a > 5'),
+ cum_count_where(col='count_2', filters='a > 0 && a < 5'),
+ cum_count_where(col='count_3', filters=['a > 0', 'a < 5']),
+ ]
+
# Rolling Operators list shared with test_rolling_ops / test_rolling_ops_proxy
cls.rolling_ops = [
# rolling sum
@@ -168,6 +176,11 @@ def setUpClass(cls) -> None:
rolling_formula_time(formula="formula_be=sum(b) + sum(e)", ts_col="Timestamp", rev_time="PT00:00:10"),
rolling_formula_time(formula="formula_be=avg(b) + avg(e)", ts_col="Timestamp", rev_time=10_000_000_000, fwd_time=-10_000_000_00),
rolling_formula_time(formula="formula_be=sum(b) + sum(b)", ts_col="Timestamp", rev_time="PT30S", fwd_time="-PT00:00:20"),
+ rolling_count_where_tick(col="count_1", filters="a > 50", rev_ticks=10),
+ rolling_count_where_tick(col="count_2", filters=["a > 0", "a <= 50"], rev_ticks=10, fwd_ticks=10),
+ rolling_count_where_time(col="count_3", filters="a > 50", ts_col="Timestamp", rev_time="PT00:00:10"),
+ rolling_count_where_time(col="count_4", filters="a > 0 && a <= 50", ts_col="Timestamp", rev_time=10_000_000_000, fwd_time=-10_000_000_00),
+ rolling_count_where_time(col="count_5", filters="a < 0 || a > 50", ts_col="Timestamp", rev_time="PT30S", fwd_time="-PT00:00:20"),
]
@@ -232,6 +245,34 @@ def test_simple_ops_proxy(self):
with update_graph.exclusive_lock(self.test_update_graph):
self.assertEqual(ct.size, rct.size)
+ def test_simple_ops_one_output(self):
+ for op in self.simple_ops_one_output:
+ with self.subTest(op):
+ for t in (self.static_table, self.ticking_table):
+ rt = t.update_by(ops=op, by="e")
+ self.assertTrue(rt.is_refreshing is t.is_refreshing)
+ self.assertEqual(len(rt.definition), 1 + len(t.definition))
+ with update_graph.exclusive_lock(self.test_update_graph):
+ self.assertEqual(rt.size, t.size)
+
+ def test_simple_ops_one_output_proxy(self):
+ pt_proxies = [self.static_table.partition_by("c").proxy(),
+ self.ticking_table.partition_by("c").proxy(),
+ ]
+
+ for op in self.simple_ops_one_output:
+ with self.subTest(op):
+ for pt_proxy in pt_proxies:
+ rt_proxy = pt_proxy.update_by(ops=op, by="e")
+
+ self.assertTrue(rt_proxy.is_refreshing is pt_proxy.is_refreshing)
+ self.assertEqual(len(rt_proxy.target.constituent_table_columns),
+ 1 + len(pt_proxy.target.constituent_table_columns))
+
+ for ct, rct in zip(pt_proxy.target.constituent_tables, rt_proxy.target.constituent_tables):
+ with update_graph.exclusive_lock(self.test_update_graph):
+ self.assertEqual(ct.size, rct.size)
+
def test_rolling_ops(self):
# Test rolling operators that produce 2 output columns
for op in self.rolling_ops:
@@ -293,5 +334,55 @@ def test_multiple_ops(self):
with update_graph.exclusive_lock(self.test_update_graph):
self.assertEqual(rt.size, t.size)
+ def test_cum_count_where_output(self):
+ """
+ Test and validation of the cum_count_where feature
+ """
+ test_table = empty_table(4).update(["a=ii", "b=ii%2"])
+ count_aggs = [
+ cum_count_where(col="count1", filters="a >= 1"),
+ cum_count_where(col="count2", filters="a >= 1 && b == 0"),
+ ]
+ result_table = test_table.update_by(ops=count_aggs)
+ self.assertEqual(result_table.size, 4)
+
+ # get the table as a local pandas dataframe
+ df = to_pandas(result_table)
+ # assert the values meet expectations
+ self.assertEqual(df.loc[0, "count1"], 0)
+ self.assertEqual(df.loc[1, "count1"], 1)
+ self.assertEqual(df.loc[2, "count1"], 2)
+ self.assertEqual(df.loc[3, "count1"], 3)
+
+ self.assertEqual(df.loc[0, "count2"], 0)
+ self.assertEqual(df.loc[1, "count2"], 0)
+ self.assertEqual(df.loc[2, "count2"], 1)
+ self.assertEqual(df.loc[3, "count2"], 1)
+
+ def test_rolling_count_where_output(self):
+ """
+ Test and validation of the rolling_count_where feature
+ """
+ test_table = empty_table(4).update(["a=ii", "b=ii%2"])
+ count_aggs = [
+ rolling_count_where_tick(col="count1", filters="a >= 1", rev_ticks=2),
+ rolling_count_where_tick(col="count2", filters="a >= 1 && b == 0", rev_ticks=2),
+ ]
+ result_table = test_table.update_by(ops=count_aggs)
+ self.assertEqual(result_table.size, 4)
+
+ # get the table as a local pandas dataframe
+ df = to_pandas(result_table)
+ # assert the values meet expectations
+ self.assertEqual(df.loc[0, "count1"], 0)
+ self.assertEqual(df.loc[1, "count1"], 1)
+ self.assertEqual(df.loc[2, "count1"], 2)
+ self.assertEqual(df.loc[3, "count1"], 2)
+
+ self.assertEqual(df.loc[0, "count2"], 0)
+ self.assertEqual(df.loc[1, "count2"], 0)
+ self.assertEqual(df.loc[2, "count2"], 1)
+ self.assertEqual(df.loc[3, "count2"], 1)
+
if __name__ == '__main__':
unittest.main()
diff --git a/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java b/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java
index 7d5f5a2acd7..ed751266d53 100644
--- a/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java
+++ b/server/src/main/java/io/deephaven/server/table/ops/UpdateByGrpcImpl.java
@@ -140,6 +140,8 @@ private static UpdateBySpec adaptSpec(UpdateByColumn.UpdateBySpec spec) {
return adaptMax(spec.getMax());
case PRODUCT:
return adaptProduct(spec.getProduct());
+ case COUNT_WHERE:
+ return adaptCountWhere(spec.getCountWhere());
case FILL:
return adaptFill(spec.getFill());
case EMA:
@@ -175,6 +177,8 @@ private static UpdateBySpec adaptSpec(UpdateByColumn.UpdateBySpec spec) {
return adaptRollingWAvg(spec.getRollingWavg());
case ROLLING_FORMULA:
return adaptRollingFormula(spec.getRollingFormula());
+ case ROLLING_COUNT_WHERE:
+ return adaptRollingCountWhere(spec.getRollingCountWhere());
case TYPE_NOT_SET:
default:
@@ -198,6 +202,11 @@ private static CumProdSpec adaptProduct(@SuppressWarnings("unused") UpdateByCumu
return CumProdSpec.of();
}
+ private static CumCountWhereSpec adaptCountWhere(
+ @SuppressWarnings("unused") UpdateByCumulativeCountWhere countWhere) {
+ return CumCountWhereSpec.of(countWhere.getResultColumn(), countWhere.getFiltersList().toArray(String[]::new));
+ }
+
private static FillBySpec adaptFill(@SuppressWarnings("unused") UpdateByFill fill) {
return FillBySpec.of();
}
@@ -328,6 +337,15 @@ private static RollingFormulaSpec adaptRollingFormula(UpdateByRollingFormula for
formula.getParamToken());
}
+ private static RollingCountWhereSpec adaptRollingCountWhere(
+ @SuppressWarnings("unused") UpdateByRollingCountWhere countWhere) {
+ return RollingCountWhereSpec.of(
+ adaptWindowScale(countWhere.getReverseWindowScale()),
+ adaptWindowScale(countWhere.getForwardWindowScale()),
+ countWhere.getResultColumn(),
+ countWhere.getFiltersList().toArray(String[]::new));
+ }
+
private static MathContext adaptMathContext(io.deephaven.proto.backplane.grpc.MathContext bigValueContext) {
return new MathContext(bigValueContext.getPrecision(), adaptRoundingMode(bigValueContext.getRoundingMode()));
}
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 b27f487aecf..5e63ba08b4b 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
@@ -230,6 +230,7 @@ static Count AggCount(String resultColumn) {
* values that pass the supplied {@code filters}.
*
* @param resultColumn The {@link Count#column() output column} name
+ * @param filters The filters to apply to the input columns
* @return The aggregation
*/
static CountWhere AggCountWhere(String resultColumn, String... filters) {
@@ -241,6 +242,7 @@ static CountWhere AggCountWhere(String resultColumn, String... filters) {
* values that pass the supplied {@code filter}.
*
* @param resultColumn The {@link Count#column() output column} name
+ * @param filter The {@link Filter} to apply to the input columns
* @return The aggregation
*/
static CountWhere AggCountWhere(String resultColumn, Filter filter) {
diff --git a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java
index baab6690ce0..ca76d9f5bf0 100644
--- a/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java
+++ b/table-api/src/main/java/io/deephaven/api/updateby/UpdateByOperation.java
@@ -4,6 +4,7 @@
package io.deephaven.api.updateby;
import io.deephaven.api.Pair;
+import io.deephaven.api.filter.Filter;
import io.deephaven.api.updateby.spec.*;
import java.time.Duration;
@@ -76,6 +77,30 @@ static UpdateByOperation CumMax(String... pairs) {
return CumMinMaxSpec.of(true).clause(pairs);
}
+ /**
+ * Create a {@link CumCountWhereSpec cumulative count where} that counts the number of rows that pass the provided
+ * filters.
+ *
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation CumCountWhere(String resultColumn, String... filters) {
+ return CumCountWhereSpec.of(resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link CumCountWhereSpec cumulative count where} that counts the number of rows that pass the provided
+ * filter.
+ *
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation CumCountWhere(String resultColumn, Filter filter) {
+ return CumCountWhereSpec.of(resultColumn, filter).clause();
+ }
+
/**
* Create a {@link FillBySpec forward fill} for the supplied column name pairs.
*
@@ -1622,6 +1647,258 @@ static UpdateByOperation RollingCount(String timestampCol, long revTime, long fw
}
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filters,
+ * using ticks as the windowing unit. Ticks are row counts and you may specify the previous window in number of rows
+ * to include. The current row is considered to belong to the reverse window, so calling this with
+ * {@code revTicks = 1} will simply return the current row. Specifying {@code revTicks = 10} will include the
+ * previous 9 rows to this one and this row for a total of 10 rows.
+ *
+ * @param revTicks the look-behind window size (in rows/ticks)
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(long revTicks, String resultColumn, String... filters) {
+ return RollingCountWhereSpec.ofTicks(revTicks, resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filters,
+ * using ticks as the windowing unit. Ticks are row counts and you may specify the previous window in number of rows
+ * to include. The current row is considered to belong to the reverse window, so calling this with
+ * {@code revTicks = 1} will simply return the current row. Specifying {@code revTicks = 10} will include the
+ * previous 9 rows to this one and this row for a total of 10 rows.
+ *
+ * @param revTicks the look-behind window size (in rows/ticks)
+ * @param fwdTicks the look-ahead window size (in rows/ticks)
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(long revTicks, long fwdTicks, String resultColumn, String... filters) {
+ return RollingCountWhereSpec.ofTicks(revTicks, fwdTicks, resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filter,
+ * using ticks as the windowing unit. Ticks are row counts and you may specify the previous window in number of rows
+ * to include. The current row is considered to belong to the reverse window, so calling this with
+ * {@code revTicks = 1} will simply return the current row. Specifying {@code revTicks = 10} will include the
+ * previous 9 rows to this one and this row for a total of 10 rows.
+ *
+ * @param revTicks the look-behind window size (in rows/ticks)
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(long revTicks, String resultColumn, Filter filter) {
+ return RollingCountWhereSpec.ofTicks(revTicks, resultColumn, filter).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filter,
+ * using ticks as the windowing unit. Ticks are row counts and you may specify the previous window in number of rows
+ * to include. The current row is considered to belong to the reverse window, so calling this with
+ * {@code revTicks = 1} will simply return the current row. Specifying {@code revTicks = 10} will include the
+ * previous 9 rows to this one and this row for a total of 10 rows.
+ *
+ * @param revTicks the look-behind window size (in rows/ticks)
+ * @param fwdTicks the look-ahead window size (in rows/ticks)
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(long revTicks, long fwdTicks, String resultColumn, Filter filter) {
+ return RollingCountWhereSpec.ofTicks(revTicks, fwdTicks, resultColumn, filter).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filters,
+ * using time as the windowing unit. This function accepts {@link Duration duration} as the reverse window
+ * parameter. A row containing a {@code null} in the timestamp column belongs to no window and will not have a value
+ * computed or be considered in the windows of other rows.
+ *
+ * Here are some examples of window values:
+ *
+ * - {@code revDuration = 0m} - contains rows that exactly match the current row timestamp
+ * - {@code revDuration = 10m} - contains rows from 10m earlier through the current row timestamp (inclusive)
+ *
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revDuration the look-behind window size (in Duration)
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, Duration revDuration, String resultColumn,
+ String... filters) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revDuration, resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filters,
+ * using time as the windowing unit. This function accepts {@link Duration durations} as the reverse and forward
+ * window parameters. Negative values are allowed and can be used to generate completely forward or completely
+ * reverse windows. A row containing a {@code null} in the timestamp column belongs to no window and will not have a
+ * value computed or be considered in the windows of other rows.
+ *
+ * Here are some examples of window values:
+ *
+ * - {@code revDuration = 0m, fwdDuration = 0m} - contains rows that exactly match the current row timestamp
+ * - {@code revDuration = 10m, fwdDuration = 0m} - contains rows from 10m earlier through the current row
+ * timestamp (inclusive)
+ * - {@code revDuration = 0m, fwdDuration = 10m} - contains rows from the current row through 10m following the
+ * current row timestamp (inclusive)
+ * - {@code revDuration = 10m, fwdDuration = 10m} - contains rows from 10m earlier through 10m following the
+ * current row timestamp (inclusive)
+ * - {@code revDuration = 10m, fwdDuration = -5m} - contains rows from 10m earlier through 5m before the current
+ * row timestamp (inclusive), this is a purely backwards looking window
+ * - {@code revDuration = -5m, fwdDuration = 10m} - contains rows from 5m following through 10m following the
+ * current row timestamp (inclusive), this is a purely forwards looking window
+ *
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revDuration the look-behind window size (in Duration)
+ * @param fwdDuration the look-ahead window size (in Duration)
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, Duration revDuration, Duration fwdDuration,
+ String resultColumn, String... filters) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revDuration, fwdDuration, resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filters,
+ * using time as the windowing unit. This function accepts {@code nanoseconds} as the reverse window parameters. A
+ * row containing a {@code null} in the timestamp column belongs to no window and will not have a value computed or
+ * be considered in the windows of other rows.
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revTime the look-behind window size (in nanoseconds)
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, long revTime, String resultColumn,
+ String... filters) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revTime, resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filters,
+ * using time as the windowing unit. This function accepts {@code nanoseconds} as the reverse and forward window
+ * parameters. Negative values are allowed and can be used to generate completely forward or completely reverse
+ * windows. A row containing a {@code null} in the timestamp column belongs to no window and will not have a value
+ * computed or be considered in the windows of other rows.
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revTime the look-behind window size (in nanoseconds)
+ * @param fwdTime the look-ahead window size (in nanoseconds)
+ * @param resultColumn The output column name in the result table
+ * @param filters The filters to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, long revTime, long fwdTime, String resultColumn,
+ String... filters) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revTime, fwdTime, resultColumn, filters).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filter,
+ * using time as the windowing unit. This function accepts {@link Duration duration} as the reverse window
+ * parameter. A row containing a {@code null} in the timestamp column belongs to no window and will not have a value
+ * computed or be considered in the windows of other rows.
+ *
+ * Here are some examples of window values:
+ *
+ * - {@code revDuration = 0m} - contains rows that exactly match the current row timestamp
+ * - {@code revDuration = 10m} - contains rows from 10m earlier through the current row timestamp (inclusive)
+ *
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revDuration the look-behind window size (in Duration)
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, Duration revDuration, String resultColumn,
+ Filter filter) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revDuration, resultColumn, filter).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filter,
+ * using time as the windowing unit. This function accepts {@link Duration durations} as the reverse and forward
+ * window parameters. Negative values are allowed and can be used to generate completely forward or completely
+ * reverse windows. A row containing a {@code null} in the timestamp column belongs to no window and will not have a
+ * value computed or be considered in the windows of other rows.
+ *
+ * Here are some examples of window values:
+ *
+ * - {@code revDuration = 0m, fwdDuration = 0m} - contains rows that exactly match the current row timestamp
+ * - {@code revDuration = 10m, fwdDuration = 0m} - contains rows from 10m earlier through the current row
+ * timestamp (inclusive)
+ * - {@code revDuration = 0m, fwdDuration = 10m} - contains rows from the current row through 10m following the
+ * current row timestamp (inclusive)
+ * - {@code revDuration = 10m, fwdDuration = 10m} - contains rows from 10m earlier through 10m following the
+ * current row timestamp (inclusive)
+ * - {@code revDuration = 10m, fwdDuration = -5m} - contains rows from 10m earlier through 5m before the current
+ * row timestamp (inclusive), this is a purely backwards looking window
+ * - {@code revDuration = -5m, fwdDuration = 10m} - contains rows from 5m following through 10m following the
+ * current row timestamp (inclusive), this is a purely forwards looking window
+ *
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revDuration the look-behind window size (in Duration)
+ * @param fwdDuration the look-ahead window size (in Duration)
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, Duration revDuration, Duration fwdDuration,
+ String resultColumn, Filter filter) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revDuration, fwdDuration, resultColumn, filter).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filter,
+ * using time as the windowing unit. This function accepts {@code nanoseconds} as the reverse window parameters. A
+ * row containing a {@code null} in the timestamp column belongs to no window and will not have a value computed or
+ * be considered in the windows of other rows.
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revTime the look-behind window size (in nanoseconds)
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, long revTime, String resultColumn, Filter filter) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revTime, resultColumn, filter).clause();
+ }
+
+ /**
+ * Create a {@link RollingCountWhereSpec rolling count where} that will count values that pass the provided filter,
+ * using time as the windowing unit. This function accepts {@code nanoseconds} as the reverse and forward window
+ * parameters. Negative values are allowed and can be used to generate completely forward or completely reverse
+ * windows. A row containing a {@code null} in the timestamp column belongs to no window and will not have a value
+ * computed or be considered in the windows of other rows.
+ *
+ * @param timestampCol the name of the timestamp column
+ * @param revTime the look-behind window size (in nanoseconds)
+ * @param fwdTime the look-ahead window size (in nanoseconds)
+ * @param resultColumn The output column name in the result table
+ * @param filter The filter to apply to the input columns
+ * @return The aggregation
+ */
+ static UpdateByOperation RollingCountWhere(String timestampCol, long revTime, long fwdTime, String resultColumn,
+ Filter filter) {
+ return RollingCountWhereSpec.ofTime(timestampCol, revTime, fwdTime, resultColumn, filter).clause();
+ }
+
+
/**
* Create a {@link RollingStdSpec rolling sample standard deviation} for the supplied column name pairs, using ticks
* as the windowing unit. Ticks are row counts and you may specify the previous window in number of rows to include.
diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/CumCountWhereSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumCountWhereSpec.java
new file mode 100644
index 00000000000..f50f63ceeda
--- /dev/null
+++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/CumCountWhereSpec.java
@@ -0,0 +1,41 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.api.updateby.spec;
+
+import io.deephaven.annotations.SimpleStyle;
+import io.deephaven.api.ColumnName;
+import io.deephaven.api.filter.Filter;
+import org.immutables.value.Value;
+import org.immutables.value.Value.Immutable;
+
+/**
+ * An {@link UpdateBySpec} for performing a windowed rolling formula operation.
+ */
+@Immutable
+@SimpleStyle
+public abstract class CumCountWhereSpec extends UpdateBySpecBase {
+ @Value.Parameter
+ public abstract ColumnName column();
+
+ @Value.Parameter
+ public abstract Filter filter();
+
+ public static CumCountWhereSpec of(String resultColumn, String... filters) {
+ return of(resultColumn, Filter.and(Filter.from(filters)));
+ }
+
+ public static CumCountWhereSpec of(String resultColumn, Filter filter) {
+ return ImmutableCumCountWhereSpec.of(ColumnName.of(resultColumn), filter);
+ }
+
+ @Override
+ public final boolean applicableTo(Class> inputType) {
+ return true;
+ }
+
+ @Override
+ public final T walk(Visitor visitor) {
+ return visitor.visit(this);
+ }
+}
diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingCountWhereSpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingCountWhereSpec.java
new file mode 100644
index 00000000000..81ef8085118
--- /dev/null
+++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/RollingCountWhereSpec.java
@@ -0,0 +1,129 @@
+//
+// Copyright (c) 2016-2024 Deephaven Data Labs and Patent Pending
+//
+package io.deephaven.api.updateby.spec;
+
+import io.deephaven.annotations.BuildableStyle;
+import io.deephaven.api.ColumnName;
+import io.deephaven.api.filter.Filter;
+import org.immutables.value.Value;
+import org.immutables.value.Value.Immutable;
+
+import java.time.Duration;
+
+/**
+ * An {@link UpdateBySpec} for performing a windowed rolling formula operation.
+ */
+@Immutable
+@BuildableStyle
+public abstract class RollingCountWhereSpec extends RollingOpSpec {
+ @Value.Parameter
+ public abstract ColumnName column();
+
+ @Value.Parameter
+ public abstract Filter filter();
+
+ public static RollingCountWhereSpec ofTicks(long revTicks, String resultColumn, String... filters) {
+ return of(WindowScale.ofTicks(revTicks), resultColumn, filters);
+ }
+
+ public static RollingCountWhereSpec ofTicks(long revTicks, long fwdTicks, String resultColumn, String... filters) {
+ return of(WindowScale.ofTicks(revTicks), WindowScale.ofTicks(fwdTicks), resultColumn, filters);
+ }
+
+ public static RollingCountWhereSpec ofTicks(long revTicks, String resultColumn, Filter filter) {
+ return of(WindowScale.ofTicks(revTicks), resultColumn, filter);
+ }
+
+ public static RollingCountWhereSpec ofTicks(long revTicks, long fwdTicks, String resultColumn, Filter filter) {
+ return of(WindowScale.ofTicks(revTicks), WindowScale.ofTicks(fwdTicks), resultColumn, filter);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, Duration revDuration, String resultColumn,
+ String... filters) {
+ return of(WindowScale.ofTime(timestampCol, revDuration), resultColumn, filters);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, Duration revDuration, Duration fwdDuration,
+ String resultColumn, String... filters) {
+ return of(WindowScale.ofTime(timestampCol, revDuration),
+ WindowScale.ofTime(timestampCol, fwdDuration),
+ resultColumn, filters);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, Duration revDuration, String resultColumn,
+ Filter filter) {
+ return of(WindowScale.ofTime(timestampCol, revDuration), resultColumn, filter);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, Duration revDuration, Duration fwdDuration,
+ String resultColumn, Filter filter) {
+ return of(WindowScale.ofTime(timestampCol, revDuration),
+ WindowScale.ofTime(timestampCol, fwdDuration),
+ resultColumn, filter);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, long revDuration, String resultColumn,
+ String... filters) {
+ return of(WindowScale.ofTime(timestampCol, revDuration),
+ resultColumn, filters);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, long revDuration, long fwdDuration,
+ String resultColumn, String... filters) {
+ return of(WindowScale.ofTime(timestampCol, revDuration),
+ WindowScale.ofTime(timestampCol, fwdDuration),
+ resultColumn, filters);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, long revDuration, String resultColumn,
+ Filter filter) {
+ return of(WindowScale.ofTime(timestampCol, revDuration), resultColumn, filter);
+ }
+
+ public static RollingCountWhereSpec ofTime(final String timestampCol, long revDuration, long fwdDuration,
+ String resultColumn, Filter filter) {
+ return of(WindowScale.ofTime(timestampCol, revDuration),
+ WindowScale.ofTime(timestampCol, fwdDuration),
+ resultColumn, filter);
+ }
+
+ // Base methods for creating the RollingFormulaSpec
+
+ public static RollingCountWhereSpec of(WindowScale revWindowScale, String resultColumn, String... filters) {
+ return of(revWindowScale, resultColumn, Filter.and(Filter.from(filters)));
+ }
+
+ public static RollingCountWhereSpec of(WindowScale revWindowScale, WindowScale fwdWindowScale, String resultColumn,
+ String... filters) {
+ return of(revWindowScale, fwdWindowScale, resultColumn, Filter.and(Filter.from(filters)));
+ }
+
+ public static RollingCountWhereSpec of(WindowScale revWindowScale, String resultColumn, Filter filter) {
+ return ImmutableRollingCountWhereSpec.builder()
+ .revWindowScale(revWindowScale)
+ .column(ColumnName.of(resultColumn))
+ .filter(filter)
+ .build();
+ }
+
+ public static RollingCountWhereSpec of(WindowScale revWindowScale, WindowScale fwdWindowScale, String resultColumn,
+ Filter filter) {
+ return ImmutableRollingCountWhereSpec.builder()
+ .revWindowScale(revWindowScale)
+ .fwdWindowScale(fwdWindowScale)
+ .column(ColumnName.of(resultColumn))
+ .filter(filter)
+ .build();
+ }
+
+ @Override
+ public final boolean applicableTo(Class> inputType) {
+ return true;
+ }
+
+ @Override
+ public final T walk(Visitor visitor) {
+ return visitor.visit(this);
+ }
+}
diff --git a/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java b/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java
index 1536c80f551..cef711bd976 100644
--- a/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java
+++ b/table-api/src/main/java/io/deephaven/api/updateby/spec/UpdateBySpec.java
@@ -88,6 +88,8 @@ interface Visitor {
T visit(CumProdSpec spec);
+ T visit(CumCountWhereSpec spec);
+
T visit(DeltaSpec spec);
T visit(RollingSumSpec spec);
@@ -102,11 +104,13 @@ interface Visitor {
T visit(RollingCountSpec spec);
+ T visit(RollingCountWhereSpec spec);
+
T visit(RollingStdSpec spec);
- T visit(RollingWAvgSpec w);
+ T visit(RollingWAvgSpec spec);
- T visit(RollingFormulaSpec w);
+ T visit(RollingFormulaSpec spec);
}
// endregion
}