From 7fd9f04cc849e33ad7d4298f11b9ce49aafbfb33 Mon Sep 17 00:00:00 2001 From: Teddy Ding Date: Fri, 13 Sep 2024 11:56:17 -0400 Subject: [PATCH 1/5] Stage FinalizeBlock events and emit in Precommit --- .../src/codegen/dydxprotocol/clob/query.ts | 67 +++ proto/dydxprotocol/clob/query.proto | 9 + protocol/app/app.go | 4 + protocol/lib/metrics/constants.go | 1 + protocol/lib/metrics/metric_keys.go | 3 + protocol/streaming/constants.go | 12 + .../streaming/full_node_streaming_manager.go | 170 ++++++ protocol/streaming/noop_streaming_manager.go | 25 + protocol/streaming/types/interface.go | 16 + protocol/x/clob/abci.go | 11 + .../clob/keeper/grpc_stream_finalize_block.go | 49 ++ protocol/x/clob/keeper/process_operations.go | 34 +- protocol/x/clob/types/query.pb.go | 559 ++++++++++++++---- protocol/x/subaccounts/keeper/subaccount.go | 6 +- 14 files changed, 816 insertions(+), 150 deletions(-) create mode 100644 protocol/streaming/constants.go create mode 100644 protocol/x/clob/keeper/grpc_stream_finalize_block.go diff --git a/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts b/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts index 41ca0c872a..31cae2a8a3 100644 --- a/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts +++ b/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts @@ -319,6 +319,18 @@ export interface StreamUpdateSDKType { taker_order?: StreamTakerOrderSDKType; subaccount_update?: StreamSubaccountUpdateSDKType; } +/** StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. */ + +export interface StagedFinalizeBlockEvent { + orderFill?: StreamOrderbookFill; + subaccountUpdate?: StreamSubaccountUpdate; +} +/** StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. */ + +export interface StagedFinalizeBlockEventSDKType { + order_fill?: StreamOrderbookFillSDKType; + subaccount_update?: StreamSubaccountUpdateSDKType; +} /** * StreamOrderbookUpdate provides information on an orderbook update. Used in * the full node GRPC stream. @@ -1396,6 +1408,61 @@ export const StreamUpdate = { }; +function createBaseStagedFinalizeBlockEvent(): StagedFinalizeBlockEvent { + return { + orderFill: undefined, + subaccountUpdate: undefined + }; +} + +export const StagedFinalizeBlockEvent = { + encode(message: StagedFinalizeBlockEvent, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.orderFill !== undefined) { + StreamOrderbookFill.encode(message.orderFill, writer.uint32(10).fork()).ldelim(); + } + + if (message.subaccountUpdate !== undefined) { + StreamSubaccountUpdate.encode(message.subaccountUpdate, writer.uint32(18).fork()).ldelim(); + } + + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): StagedFinalizeBlockEvent { + const reader = input instanceof _m0.Reader ? input : new _m0.Reader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseStagedFinalizeBlockEvent(); + + while (reader.pos < end) { + const tag = reader.uint32(); + + switch (tag >>> 3) { + case 1: + message.orderFill = StreamOrderbookFill.decode(reader, reader.uint32()); + break; + + case 2: + message.subaccountUpdate = StreamSubaccountUpdate.decode(reader, reader.uint32()); + break; + + default: + reader.skipType(tag & 7); + break; + } + } + + return message; + }, + + fromPartial(object: DeepPartial): StagedFinalizeBlockEvent { + const message = createBaseStagedFinalizeBlockEvent(); + message.orderFill = object.orderFill !== undefined && object.orderFill !== null ? StreamOrderbookFill.fromPartial(object.orderFill) : undefined; + message.subaccountUpdate = object.subaccountUpdate !== undefined && object.subaccountUpdate !== null ? StreamSubaccountUpdate.fromPartial(object.subaccountUpdate) : undefined; + return message; + } + +}; + function createBaseStreamOrderbookUpdate(): StreamOrderbookUpdate { return { snapshot: false, diff --git a/proto/dydxprotocol/clob/query.proto b/proto/dydxprotocol/clob/query.proto index 9d47ea4641..fb4757e875 100644 --- a/proto/dydxprotocol/clob/query.proto +++ b/proto/dydxprotocol/clob/query.proto @@ -198,6 +198,15 @@ message StreamUpdate { } } +// StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. +message StagedFinalizeBlockEvent { + // Contains one of StreamOrderbookFill, StreamSubaccountUpdate. + oneof event { + StreamOrderbookFill order_fill = 1; + dydxprotocol.subaccounts.StreamSubaccountUpdate subaccount_update = 2; + } +} + // StreamOrderbookUpdate provides information on an orderbook update. Used in // the full node GRPC stream. message StreamOrderbookUpdate { diff --git a/protocol/app/app.go b/protocol/app/app.go index fbb8db6bd0..a8f2cec91e 100644 --- a/protocol/app/app.go +++ b/protocol/app/app.go @@ -469,6 +469,7 @@ func New( statsmoduletypes.TransientStoreKey, rewardsmoduletypes.TransientStoreKey, indexer_manager.TransientStoreKey, + streaming.StreamingManagerTransientStoreKey, perpetualsmoduletypes.TransientStoreKey, ) memKeys := storetypes.NewMemoryStoreKeys(capabilitytypes.MemStoreKey, clobmoduletypes.MemStoreKey) @@ -764,6 +765,7 @@ func New( appFlags, appCodec, logger, + tkeys[streaming.StreamingManagerTransientStoreKey], ) timeProvider := &timelib.TimeProviderImpl{} @@ -2059,6 +2061,7 @@ func getFullNodeStreamingManagerFromOptions( appFlags flags.Flags, cdc codec.Codec, logger log.Logger, + streamingManagerTransientStoreKey storetypes.StoreKey, ) (manager streamingtypes.FullNodeStreamingManager, wsServer *ws.WebsocketServer) { logger = logger.With(log.ModuleKey, "full-node-streaming") if appFlags.GrpcStreamingEnabled { @@ -2072,6 +2075,7 @@ func getFullNodeStreamingManagerFromOptions( appFlags.GrpcStreamingMaxBatchSize, appFlags.GrpcStreamingMaxChannelBufferSize, appFlags.FullNodeStreamingSnapshotInterval, + streamingManagerTransientStoreKey, ) // Start websocket server. diff --git a/protocol/lib/metrics/constants.go b/protocol/lib/metrics/constants.go index 0ac766bf5b..a0f851eb58 100644 --- a/protocol/lib/metrics/constants.go +++ b/protocol/lib/metrics/constants.go @@ -202,6 +202,7 @@ const ( UpdateType = "update_type" ValidateMatches = "validate_matches" ValidateOrder = "validate_order" + StreamBatchUpdatesAfterFinalizeBlock = "stream_batch_updates_after_finalize_block" // MemCLOB. AddedToOrderBook = "added_to_orderbook" diff --git a/protocol/lib/metrics/metric_keys.go b/protocol/lib/metrics/metric_keys.go index f980275d7f..2a44834332 100644 --- a/protocol/lib/metrics/metric_keys.go +++ b/protocol/lib/metrics/metric_keys.go @@ -83,6 +83,9 @@ const ( GrpcStreamNumUpdatesBuffered = "grpc_stream_num_updates_buffered" GrpcFlushUpdatesLatency = "grpc_flush_updates_latency" GrpcSubscriptionChannelLength = "grpc_subscription_channel_length" + GrpcStagedAllFinalizeBlockUpdates = "grpc_staged_all_finalize_block_updates" + GrpcStagedFillFinalizeBlockUpdates = "grpc_staged_finalize_block_fill_updates" + GrpcStagedSubaccountFinalizeBlockUpdates = "grpc_staged_finalize_block_subaccount_updates" EndBlocker = "end_blocker" EndBlockerLag = "end_blocker_lag" diff --git a/protocol/streaming/constants.go b/protocol/streaming/constants.go new file mode 100644 index 0000000000..c3c3c255a7 --- /dev/null +++ b/protocol/streaming/constants.go @@ -0,0 +1,12 @@ +package streaming + +const ( + // Transient store key for storing staged events. + StreamingManagerTransientStoreKey = "tmp_streaming" + + // Key for storing the count of staged events. + StagedEventsCountKey = "EvtCnt" + + // Key prefix for staged events. + StagedEventsKeyPrefix = "Evt:" +) diff --git a/protocol/streaming/full_node_streaming_manager.go b/protocol/streaming/full_node_streaming_manager.go index a6fe37c512..f357908f28 100644 --- a/protocol/streaming/full_node_streaming_manager.go +++ b/protocol/streaming/full_node_streaming_manager.go @@ -1,15 +1,21 @@ package streaming import ( + "encoding/binary" "fmt" "sync" "sync/atomic" "time" + "github.com/dydxprotocol/v4-chain/protocol/lib" satypes "github.com/dydxprotocol/v4-chain/protocol/x/subaccounts/types" "cosmossdk.io/log" + "cosmossdk.io/store/prefix" + storetypes "cosmossdk.io/store/types" + "github.com/cosmos/cosmos-sdk/telemetry" sdk "github.com/cosmos/cosmos-sdk/types" + ante_types "github.com/dydxprotocol/v4-chain/protocol/app/ante/types" "github.com/dydxprotocol/v4-chain/protocol/lib/metrics" "github.com/dydxprotocol/v4-chain/protocol/streaming/types" streaming_util "github.com/dydxprotocol/v4-chain/protocol/streaming/util" @@ -50,6 +56,9 @@ type FullNodeStreamingManagerImpl struct { // Block interval in which snapshot info should be sent out in. // Defaults to 0, which means only one snapshot will be sent out. snapshotBlockInterval uint32 + + // stores the staged FinalizeBlock events for full node streaming. + streamingManagerTransientStoreKey storetypes.StoreKey } // OrderbookSubscription represents a active subscription to the orderbook updates stream. @@ -86,6 +95,7 @@ func NewFullNodeStreamingManager( maxUpdatesInCache uint32, maxSubscriptionChannelSize uint32, snapshotBlockInterval uint32, + streamingManagerTransientStoreKey storetypes.StoreKey, ) *FullNodeStreamingManagerImpl { fullNodeStreamingManager := &FullNodeStreamingManagerImpl{ logger: logger, @@ -102,6 +112,8 @@ func NewFullNodeStreamingManager( maxUpdatesInCache: maxUpdatesInCache, maxSubscriptionChannelSize: maxSubscriptionChannelSize, snapshotBlockInterval: snapshotBlockInterval, + + streamingManagerTransientStoreKey: streamingManagerTransientStoreKey, } // Start the goroutine for pushing order updates through. @@ -367,6 +379,84 @@ func (sm *FullNodeStreamingManagerImpl) sendStreamUpdates( } } +func getStagedEventsCount(store storetypes.KVStore) uint32 { + countsBytes := store.Get([]byte(StagedEventsCountKey)) + if countsBytes == nil { + return 0 + } + return binary.BigEndian.Uint32(countsBytes) +} + +// Stage a subaccount update event in transient store, during `FinalizeBlock`. +func (sm *FullNodeStreamingManagerImpl) StageFinalizeBlockSubaccountUpdate( + ctx sdk.Context, + subaccountUpdate satypes.StreamSubaccountUpdate, +) { + stagedEvent := clobtypes.StagedFinalizeBlockEvent{ + Event: &clobtypes.StagedFinalizeBlockEvent_SubaccountUpdate{ + SubaccountUpdate: &subaccountUpdate, + }, + } + sm.stageFinalizeBlockEvent( + ctx, + clobtypes.Amino.MustMarshal(stagedEvent), + ) +} + +// Stage a fill event in transient store, during `FinalizeBlock`. +func (sm *FullNodeStreamingManagerImpl) StageFinalizeBlockFill( + ctx sdk.Context, + fill clobtypes.StreamOrderbookFill, +) { + stagedEvent := clobtypes.StagedFinalizeBlockEvent{ + Event: &clobtypes.StagedFinalizeBlockEvent_OrderFill{ + OrderFill: &fill, + }, + } + sm.stageFinalizeBlockEvent( + ctx, + clobtypes.Amino.MustMarshal(stagedEvent), + ) +} + +func getStagedFinalizeBlockEvents(store storetypes.KVStore) []clobtypes.StagedFinalizeBlockEvent { + count := getStagedEventsCount(store) + events := make([]clobtypes.StagedFinalizeBlockEvent, count) + store = prefix.NewStore(store, []byte(StagedEventsKeyPrefix)) + for i := uint32(0); i < count; i++ { + var event clobtypes.StagedFinalizeBlockEvent + bytes := store.Get(lib.Uint32ToKey(i)) + clobtypes.Amino.MustUnmarshal(bytes, &event) + events[i] = event + } + return events +} + +// Retrieve all events staged during `FinalizeBlock`. +func (sm *FullNodeStreamingManagerImpl) GetStagedFinalizeBlockEvents( + ctx sdk.Context, +) []clobtypes.StagedFinalizeBlockEvent { + noGasCtx := ctx.WithGasMeter(ante_types.NewFreeInfiniteGasMeter()) + store := noGasCtx.TransientStore(sm.streamingManagerTransientStoreKey) + return getStagedFinalizeBlockEvents(store) +} + +func (sm *FullNodeStreamingManagerImpl) stageFinalizeBlockEvent( + ctx sdk.Context, + eventBytes []byte, +) { + noGasCtx := ctx.WithGasMeter(ante_types.NewFreeInfiniteGasMeter()) + store := noGasCtx.TransientStore(sm.streamingManagerTransientStoreKey) + + // Increment events count. + count := getStagedEventsCount(store) + store.Set([]byte(StagedEventsCountKey), lib.Uint32ToKey(count+1)) + + // Store events keyed by index. + store = prefix.NewStore(store, []byte(StagedEventsKeyPrefix)) + store.Set(lib.Uint32ToKey(count), eventBytes) +} + // SendCombinedSnapshot sends messages to a particular subscriber without buffering. // Note this method requires the lock and assumes that the lock has already been // acquired by the caller. @@ -703,6 +793,86 @@ func (sm *FullNodeStreamingManagerImpl) GetSubaccountSnapshotsForInitStreams( return ret } +// Grpc Streaming logic after consensus agrees on a block. +// - Stream all events staged during `FinalizeBlock`. +// - Stream orderbook updates to sync fills in local ops queue. +func (sm *FullNodeStreamingManagerImpl) StreamBatchUpdatesAfterFinalizeBlock( + ctx sdk.Context, + orderBookUpdatesToSyncLocalOpsQueue *clobtypes.OffchainUpdates, + perpetualIdToClobPairId map[uint32][]clobtypes.ClobPairId, +) { + // Flush all pending updates, since we want the onchain updates to arrive in a batch. + sm.FlushStreamUpdates() + + finalizedFills, finalizedSubaccountUpdates := sm.getStagedEventsFromFinalizeBlock(ctx) + + // TODO(CT-1190): Stream below in a single batch. + // Send orderbook updates to sync optimistic orderbook onchain state after FinalizeBlock. + sm.SendOrderbookUpdates( + orderBookUpdatesToSyncLocalOpsQueue, + uint32(ctx.BlockHeight()), + ctx.ExecMode(), + ) + + // Send finalized fills from FinalizeBlock. + sm.SendOrderbookFillUpdates( + finalizedFills, + uint32(ctx.BlockHeight()), + ctx.ExecMode(), + perpetualIdToClobPairId, + ) + + // Send finalized subaccount updates from FinalizeBlock. + sm.SendFinalizedSubaccountUpdates( + finalizedSubaccountUpdates, + uint32(ctx.BlockHeight()), + ctx.ExecMode(), + ) +} + +// getStagedEventsFromFinalizeBlock returns staged events from `FinalizeBlock`. +// It should be called after the consensus agrees on a block (e.g. Precommitter). +func (sm *FullNodeStreamingManagerImpl) getStagedEventsFromFinalizeBlock( + ctx sdk.Context, +) ( + finalizedFills []clobtypes.StreamOrderbookFill, + finalizedSubaccountUpdates []satypes.StreamSubaccountUpdate, +) { + // Get onchain stream events stored in transient store. + stagedEvents := sm.GetStagedFinalizeBlockEvents(ctx) + + telemetry.SetGauge( + float32(len(stagedEvents)), + types.ModuleName, + metrics.GrpcStagedAllFinalizeBlockUpdates, + metrics.Count, + ) + + for _, stagedEvent := range stagedEvents { + switch event := stagedEvent.Event.(type) { + case *clobtypes.StagedFinalizeBlockEvent_OrderFill: + finalizedFills = append(finalizedFills, *event.OrderFill) + case *clobtypes.StagedFinalizeBlockEvent_SubaccountUpdate: + finalizedSubaccountUpdates = append(finalizedSubaccountUpdates, *event.SubaccountUpdate) + } + } + + telemetry.SetGauge( + float32(len(finalizedSubaccountUpdates)), + types.ModuleName, + metrics.GrpcStagedSubaccountFinalizeBlockUpdates, + metrics.Count, + ) + telemetry.SetGauge( + float32(len(finalizedFills)), + types.ModuleName, + metrics.GrpcStagedFillFinalizeBlockUpdates, + metrics.Count, + ) + + return finalizedFills, finalizedSubaccountUpdates +} + func (sm *FullNodeStreamingManagerImpl) InitializeNewStreams( getOrderbookSnapshot func(clobPairId clobtypes.ClobPairId) *clobtypes.OffchainUpdates, subaccountSnapshots map[satypes.SubaccountId]*satypes.StreamSubaccountUpdate, diff --git a/protocol/streaming/noop_streaming_manager.go b/protocol/streaming/noop_streaming_manager.go index 6e9c00895d..4df60bc427 100644 --- a/protocol/streaming/noop_streaming_manager.go +++ b/protocol/streaming/noop_streaming_manager.go @@ -78,3 +78,28 @@ func (sm *NoopGrpcStreamingManager) InitializeNewStreams( func (sm *NoopGrpcStreamingManager) Stop() { } + +func (sm *NoopGrpcStreamingManager) StageFinalizeBlockFill( + ctx sdk.Context, + fill clobtypes.StreamOrderbookFill, +) { +} + +func (sm *NoopGrpcStreamingManager) GetStagedFinalizeBlockEvents( + ctx sdk.Context, +) []clobtypes.StagedFinalizeBlockEvent { + return nil +} + +func (sm *NoopGrpcStreamingManager) StageFinalizeBlockSubaccountUpdate( + ctx sdk.Context, + subaccountUpdate satypes.StreamSubaccountUpdate, +) { +} + +func (sm *NoopGrpcStreamingManager) StreamBatchUpdatesAfterFinalizeBlock( + ctx sdk.Context, + orderBookUpdatesToSyncLocalOpsQueue *clobtypes.OffchainUpdates, + perpetualIdToClobPairId map[uint32][]clobtypes.ClobPairId, +) { +} diff --git a/protocol/streaming/types/interface.go b/protocol/streaming/types/interface.go index e3dff9d94b..0f097d3e75 100644 --- a/protocol/streaming/types/interface.go +++ b/protocol/streaming/types/interface.go @@ -50,7 +50,23 @@ type FullNodeStreamingManager interface { blockHeight uint32, execMode sdk.ExecMode, ) + StageFinalizeBlockFill( + ctx sdk.Context, + fill clobtypes.StreamOrderbookFill, + ) + StageFinalizeBlockSubaccountUpdate( + ctx sdk.Context, + subaccountUpdate satypes.StreamSubaccountUpdate, + ) + GetStagedFinalizeBlockEvents( + ctx sdk.Context, + ) []clobtypes.StagedFinalizeBlockEvent TracksSubaccountId(id satypes.SubaccountId) bool + StreamBatchUpdatesAfterFinalizeBlock( + ctx sdk.Context, + orderBookUpdatesToSyncLocalOpsQueue *clobtypes.OffchainUpdates, + perpetualIdToClobPairId map[uint32][]clobtypes.ClobPairId, + ) } type OutgoingMessageSender interface { diff --git a/protocol/x/clob/abci.go b/protocol/x/clob/abci.go index 96bacb2cc5..9fd5f4a0b5 100644 --- a/protocol/x/clob/abci.go +++ b/protocol/x/clob/abci.go @@ -46,6 +46,17 @@ func BeginBlocker( keeper.ResetAllDeliveredOrderIds(ctx) } +// Precommit executes all ABCI Precommit logic respective to the clob module. +func Precommit( + ctx sdk.Context, + keeper keeper.Keeper, +) { + if streamingManager := keeper.GetFullNodeStreamingManager(); !streamingManager.Enabled() { + return + } + keeper.StreamBatchUpdatesAfterFinalizeBlock(ctx) +} + // EndBlocker executes all ABCI EndBlock logic respective to the clob module. func EndBlocker( ctx sdk.Context, diff --git a/protocol/x/clob/keeper/grpc_stream_finalize_block.go b/protocol/x/clob/keeper/grpc_stream_finalize_block.go new file mode 100644 index 0000000000..6216eba461 --- /dev/null +++ b/protocol/x/clob/keeper/grpc_stream_finalize_block.go @@ -0,0 +1,49 @@ +package keeper + +import ( + "time" + + "github.com/cosmos/cosmos-sdk/telemetry" + sdk "github.com/cosmos/cosmos-sdk/types" + "github.com/dydxprotocol/v4-chain/protocol/lib/metrics" + "github.com/dydxprotocol/v4-chain/protocol/x/clob/types" +) + +// Returns the order updates needed to update the fill amount for the orders +// from local ops queue, according to the latest onchain state (after FinalizeBlock). +func (k Keeper) getUpdatesToSyncLocalOpsQueue( + ctx sdk.Context, +) *types.OffchainUpdates { + localValidatorOperationsQueue, _ := k.MemClob.GetOperationsToReplay(ctx) + fetchOrdersInvolvedInOpQueue(localValidatorOperationsQueue) + orderIdsFromLocal := fetchOrdersInvolvedInOpQueue( + localValidatorOperationsQueue, + ) + allUpdates := types.NewOffchainUpdates() + for orderId := range orderIdsFromLocal { + orderbookUpdate := k.MemClob.GetOrderbookUpdatesForOrderUpdate(ctx, orderId) + allUpdates.Append(orderbookUpdate) + } + return allUpdates +} + +// Grpc Streaming logic after consensus agrees on a block. +// - Stream all events staged during `FinalizeBlock`. +// - Stream orderbook updates to sync fills in local ops queue. +func (k Keeper) StreamBatchUpdatesAfterFinalizeBlock( + ctx sdk.Context, +) { + defer telemetry.MeasureSince( + time.Now(), + types.ModuleName, + metrics.StreamBatchUpdatesAfterFinalizeBlock, + metrics.Latency, + ) + orderBookUpdatesToSyncLocalOpsQueue := k.getUpdatesToSyncLocalOpsQueue(ctx) + + k.GetFullNodeStreamingManager().StreamBatchUpdatesAfterFinalizeBlock( + ctx, + orderBookUpdatesToSyncLocalOpsQueue, + k.PerpetualIdToClobPairId, + ) +} diff --git a/protocol/x/clob/keeper/process_operations.go b/protocol/x/clob/keeper/process_operations.go index d202c2a270..cddb5fb68b 100644 --- a/protocol/x/clob/keeper/process_operations.go +++ b/protocol/x/clob/keeper/process_operations.go @@ -57,26 +57,6 @@ func (k Keeper) ProcessProposerOperations( return errorsmod.Wrapf(types.ErrInvalidMsgProposedOperations, "Error: %+v", err) } - // If grpc streams are on, send absolute fill amounts from local + proposed opqueue to the grpc stream. - // This effetively reverts the optimitic orderbook updates during CheckTx. - if streamingManager := k.GetFullNodeStreamingManager(); streamingManager.Enabled() { - localValidatorOperationsQueue, _ := k.MemClob.GetOperationsToReplay(ctx) - orderIdsFromProposed := fetchOrdersInvolvedInOpQueue( - operations, - ) - orderIdsFromLocal := fetchOrdersInvolvedInOpQueue( - localValidatorOperationsQueue, - ) - orderIdSetToUpdate := lib.MergeMaps(orderIdsFromLocal, orderIdsFromProposed) - - allUpdates := types.NewOffchainUpdates() - for orderId := range orderIdSetToUpdate { - orderbookUpdate := k.MemClob.GetOrderbookUpdatesForOrderUpdate(ctx, orderId) - allUpdates.Append(orderbookUpdate) - } - k.SendOrderbookUpdates(ctx, allUpdates) - } - log.DebugLog(ctx, "Processing operations queue", log.OperationsQueue, types.GetInternalOperationsQueueTextString(operations)) @@ -550,6 +530,7 @@ func (k Keeper) PersistMatchOrdersToState( // if GRPC streaming is on, emit a generated clob match to stream. if streamingManager := k.GetFullNodeStreamingManager(); streamingManager.Enabled() { + // Note: GenerateStreamOrderbookFill doesn't rely on MemClob state. streamOrderbookFill := k.MemClob.GenerateStreamOrderbookFill( ctx, types.ClobMatch{ @@ -560,11 +541,10 @@ func (k Keeper) PersistMatchOrdersToState( &takerOrder, makerOrders, ) - k.SendOrderbookFillUpdates( + + k.GetFullNodeStreamingManager().StageFinalizeBlockFill( ctx, - []types.StreamOrderbookFill{ - streamOrderbookFill, - }, + streamOrderbookFill, ) } @@ -669,11 +649,9 @@ func (k Keeper) PersistMatchLiquidationToState( takerOrder, makerOrders, ) - k.SendOrderbookFillUpdates( + k.GetFullNodeStreamingManager().StageFinalizeBlockFill( ctx, - []types.StreamOrderbookFill{ - streamOrderbookFill, - }, + streamOrderbookFill, ) } return nil diff --git a/protocol/x/clob/types/query.pb.go b/protocol/x/clob/types/query.pb.go index 4f4d01af14..279280a0ae 100644 --- a/protocol/x/clob/types/query.pb.go +++ b/protocol/x/clob/types/query.pb.go @@ -1004,6 +1004,95 @@ func (*StreamUpdate) XXX_OneofWrappers() []interface{} { } } +// StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. +type StagedFinalizeBlockEvent struct { + // Contains one of StreamOrderbookFill, StreamSubaccountUpdate. + // + // Types that are valid to be assigned to Event: + // + // *StagedFinalizeBlockEvent_OrderFill + // *StagedFinalizeBlockEvent_SubaccountUpdate + Event isStagedFinalizeBlockEvent_Event `protobuf_oneof:"event"` +} + +func (m *StagedFinalizeBlockEvent) Reset() { *m = StagedFinalizeBlockEvent{} } +func (m *StagedFinalizeBlockEvent) String() string { return proto.CompactTextString(m) } +func (*StagedFinalizeBlockEvent) ProtoMessage() {} +func (*StagedFinalizeBlockEvent) Descriptor() ([]byte, []int) { + return fileDescriptor_3365c195b25c5bc0, []int{17} +} +func (m *StagedFinalizeBlockEvent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StagedFinalizeBlockEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StagedFinalizeBlockEvent.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *StagedFinalizeBlockEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_StagedFinalizeBlockEvent.Merge(m, src) +} +func (m *StagedFinalizeBlockEvent) XXX_Size() int { + return m.Size() +} +func (m *StagedFinalizeBlockEvent) XXX_DiscardUnknown() { + xxx_messageInfo_StagedFinalizeBlockEvent.DiscardUnknown(m) +} + +var xxx_messageInfo_StagedFinalizeBlockEvent proto.InternalMessageInfo + +type isStagedFinalizeBlockEvent_Event interface { + isStagedFinalizeBlockEvent_Event() + MarshalTo([]byte) (int, error) + Size() int +} + +type StagedFinalizeBlockEvent_OrderFill struct { + OrderFill *StreamOrderbookFill `protobuf:"bytes,1,opt,name=order_fill,json=orderFill,proto3,oneof" json:"order_fill,omitempty"` +} +type StagedFinalizeBlockEvent_SubaccountUpdate struct { + SubaccountUpdate *types.StreamSubaccountUpdate `protobuf:"bytes,2,opt,name=subaccount_update,json=subaccountUpdate,proto3,oneof" json:"subaccount_update,omitempty"` +} + +func (*StagedFinalizeBlockEvent_OrderFill) isStagedFinalizeBlockEvent_Event() {} +func (*StagedFinalizeBlockEvent_SubaccountUpdate) isStagedFinalizeBlockEvent_Event() {} + +func (m *StagedFinalizeBlockEvent) GetEvent() isStagedFinalizeBlockEvent_Event { + if m != nil { + return m.Event + } + return nil +} + +func (m *StagedFinalizeBlockEvent) GetOrderFill() *StreamOrderbookFill { + if x, ok := m.GetEvent().(*StagedFinalizeBlockEvent_OrderFill); ok { + return x.OrderFill + } + return nil +} + +func (m *StagedFinalizeBlockEvent) GetSubaccountUpdate() *types.StreamSubaccountUpdate { + if x, ok := m.GetEvent().(*StagedFinalizeBlockEvent_SubaccountUpdate); ok { + return x.SubaccountUpdate + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*StagedFinalizeBlockEvent) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*StagedFinalizeBlockEvent_OrderFill)(nil), + (*StagedFinalizeBlockEvent_SubaccountUpdate)(nil), + } +} + // StreamOrderbookUpdate provides information on an orderbook update. Used in // the full node GRPC stream. type StreamOrderbookUpdate struct { @@ -1021,7 +1110,7 @@ func (m *StreamOrderbookUpdate) Reset() { *m = StreamOrderbookUpdate{} } func (m *StreamOrderbookUpdate) String() string { return proto.CompactTextString(m) } func (*StreamOrderbookUpdate) ProtoMessage() {} func (*StreamOrderbookUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{17} + return fileDescriptor_3365c195b25c5bc0, []int{18} } func (m *StreamOrderbookUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1081,7 +1170,7 @@ func (m *StreamOrderbookFill) Reset() { *m = StreamOrderbookFill{} } func (m *StreamOrderbookFill) String() string { return proto.CompactTextString(m) } func (*StreamOrderbookFill) ProtoMessage() {} func (*StreamOrderbookFill) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{18} + return fileDescriptor_3365c195b25c5bc0, []int{19} } func (m *StreamOrderbookFill) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1152,7 +1241,7 @@ func (m *StreamTakerOrder) Reset() { *m = StreamTakerOrder{} } func (m *StreamTakerOrder) String() string { return proto.CompactTextString(m) } func (*StreamTakerOrder) ProtoMessage() {} func (*StreamTakerOrder) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{19} + return fileDescriptor_3365c195b25c5bc0, []int{20} } func (m *StreamTakerOrder) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1255,7 +1344,7 @@ func (m *StreamTakerOrderStatus) Reset() { *m = StreamTakerOrderStatus{} func (m *StreamTakerOrderStatus) String() string { return proto.CompactTextString(m) } func (*StreamTakerOrderStatus) ProtoMessage() {} func (*StreamTakerOrderStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{20} + return fileDescriptor_3365c195b25c5bc0, []int{21} } func (m *StreamTakerOrderStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1324,6 +1413,7 @@ func init() { proto.RegisterType((*StreamOrderbookUpdatesRequest)(nil), "dydxprotocol.clob.StreamOrderbookUpdatesRequest") proto.RegisterType((*StreamOrderbookUpdatesResponse)(nil), "dydxprotocol.clob.StreamOrderbookUpdatesResponse") proto.RegisterType((*StreamUpdate)(nil), "dydxprotocol.clob.StreamUpdate") + proto.RegisterType((*StagedFinalizeBlockEvent)(nil), "dydxprotocol.clob.StagedFinalizeBlockEvent") proto.RegisterType((*StreamOrderbookUpdate)(nil), "dydxprotocol.clob.StreamOrderbookUpdate") proto.RegisterType((*StreamOrderbookFill)(nil), "dydxprotocol.clob.StreamOrderbookFill") proto.RegisterType((*StreamTakerOrder)(nil), "dydxprotocol.clob.StreamTakerOrder") @@ -1333,111 +1423,114 @@ func init() { func init() { proto.RegisterFile("dydxprotocol/clob/query.proto", fileDescriptor_3365c195b25c5bc0) } var fileDescriptor_3365c195b25c5bc0 = []byte{ - // 1661 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xc1, 0x4f, 0xdc, 0x46, - 0x17, 0x5f, 0xb3, 0x84, 0xc0, 0xdb, 0x40, 0x60, 0x08, 0xc9, 0x66, 0x21, 0x0b, 0x71, 0xbe, 0x90, - 0x85, 0x7c, 0x59, 0x03, 0x89, 0xa2, 0x7c, 0xe1, 0x53, 0x3e, 0x01, 0xdf, 0x47, 0x88, 0x14, 0xbe, - 0x10, 0x43, 0x12, 0xd4, 0x46, 0xb2, 0xbc, 0xf6, 0xb0, 0x58, 0xd8, 0x9e, 0xc5, 0x1e, 0xaf, 0x40, - 0x55, 0x55, 0xa9, 0x87, 0x5c, 0xda, 0x4a, 0x91, 0x7a, 0xe8, 0xa1, 0x52, 0x2f, 0x3d, 0xf5, 0x50, - 0xa9, 0x97, 0x1e, 0xab, 0xb6, 0xb7, 0x1c, 0x23, 0xf5, 0xd2, 0x43, 0x55, 0x55, 0x49, 0xcf, 0xfd, - 0x1b, 0x2a, 0xcf, 0x8c, 0x77, 0xbd, 0xbb, 0xf6, 0x42, 0xb8, 0x80, 0xfd, 0xe6, 0xbd, 0x37, 0xbf, - 0xf7, 0xde, 0x6f, 0xde, 0x3c, 0x2f, 0x5c, 0x32, 0x0f, 0xcd, 0x83, 0x9a, 0x47, 0x28, 0x31, 0x88, - 0xad, 0x18, 0x36, 0xa9, 0x28, 0xfb, 0x01, 0xf6, 0x0e, 0xcb, 0x4c, 0x86, 0x46, 0xe2, 0xcb, 0xe5, - 0x70, 0xb9, 0x70, 0xae, 0x4a, 0xaa, 0x84, 0x89, 0x94, 0xf0, 0x89, 0x2b, 0x16, 0x26, 0xaa, 0x84, - 0x54, 0x6d, 0xac, 0xe8, 0x35, 0x4b, 0xd1, 0x5d, 0x97, 0x50, 0x9d, 0x5a, 0xc4, 0xf5, 0xc5, 0xea, - 0xac, 0x41, 0x7c, 0x87, 0xf8, 0x4a, 0x45, 0xf7, 0x31, 0xf7, 0xaf, 0xd4, 0xe7, 0x2b, 0x98, 0xea, - 0xf3, 0x4a, 0x4d, 0xaf, 0x5a, 0x2e, 0x53, 0x16, 0xba, 0x4a, 0x27, 0xa2, 0x8a, 0x4d, 0x8c, 0x3d, - 0xcd, 0xd3, 0x29, 0xd6, 0x6c, 0xcb, 0xb1, 0xa8, 0x66, 0x10, 0x77, 0xc7, 0xaa, 0x0a, 0x83, 0xcb, - 0x9d, 0x06, 0xe1, 0x1f, 0xad, 0xa6, 0x5b, 0x9e, 0x50, 0x99, 0xeb, 0x54, 0xc1, 0xfb, 0x81, 0x45, - 0x0f, 0x35, 0x6a, 0x61, 0x2f, 0xc9, 0x69, 0x42, 0x5e, 0x88, 0x67, 0xe2, 0xc8, 0xe1, 0x64, 0xe7, - 0xb2, 0xa3, 0x53, 0x63, 0x17, 0x47, 0x11, 0x5f, 0xef, 0x54, 0xb0, 0xad, 0xfd, 0xc0, 0x32, 0x79, - 0x5e, 0x5a, 0x37, 0x1b, 0x4f, 0xf0, 0x86, 0xeb, 0x62, 0xf1, 0x5e, 0xcb, 0xa2, 0xe5, 0x9a, 0xf8, - 0x00, 0x7b, 0x0a, 0xd9, 0xd9, 0xd1, 0x8c, 0x5d, 0xdd, 0x72, 0xb5, 0xa0, 0x66, 0xea, 0x14, 0xfb, - 0x9d, 0x12, 0x61, 0x5f, 0x6a, 0xb1, 0xf7, 0x83, 0x8a, 0x6e, 0x18, 0x24, 0x70, 0xa9, 0xaf, 0xf8, - 0xd4, 0xc3, 0xba, 0x63, 0xb9, 0x11, 0x8c, 0x99, 0x74, 0xcd, 0xc6, 0x33, 0x57, 0x95, 0x67, 0xe0, - 0xc2, 0xe3, 0xb0, 0x8c, 0xf7, 0x31, 0x5d, 0xb1, 0x49, 0x65, 0x43, 0xb7, 0x3c, 0x15, 0xef, 0x07, - 0xd8, 0xa7, 0x68, 0x08, 0x7a, 0x2c, 0x33, 0x2f, 0x4d, 0x49, 0xa5, 0x41, 0xb5, 0xc7, 0x32, 0xe5, - 0x67, 0x30, 0xc6, 0x54, 0x9b, 0x7a, 0x7e, 0x8d, 0xb8, 0x3e, 0x46, 0xf7, 0x60, 0xa0, 0x51, 0x27, - 0xa6, 0x9f, 0x5b, 0x18, 0x2f, 0x77, 0xf0, 0xad, 0x1c, 0xd9, 0x2d, 0xf7, 0xbe, 0xfa, 0x7d, 0x32, - 0xa3, 0xf6, 0x1b, 0xe2, 0x5d, 0xd6, 0x05, 0x86, 0x25, 0xdb, 0x6e, 0xc7, 0xb0, 0x0a, 0xd0, 0xe4, - 0x95, 0xf0, 0x3d, 0x5d, 0xe6, 0x24, 0x2c, 0x87, 0x24, 0x2c, 0x73, 0x92, 0x0b, 0x12, 0x96, 0x37, - 0xf4, 0x2a, 0x16, 0xb6, 0x6a, 0xcc, 0x52, 0xfe, 0x5a, 0x82, 0x7c, 0x0b, 0xf8, 0x25, 0xdb, 0x4e, - 0xc3, 0x9f, 0x7d, 0x47, 0xfc, 0xe8, 0x7e, 0x0b, 0xc8, 0x1e, 0x06, 0xf2, 0xda, 0x91, 0x20, 0xf9, - 0xe6, 0x2d, 0x28, 0x7f, 0x93, 0x60, 0x72, 0x1d, 0xd7, 0xff, 0x4f, 0x4c, 0xbc, 0x45, 0xc2, 0xbf, - 0x2b, 0xba, 0x6d, 0x04, 0x36, 0x5b, 0x8c, 0x32, 0xf2, 0x1c, 0xce, 0xf3, 0x53, 0x54, 0xf3, 0x48, - 0x8d, 0xf8, 0xd8, 0xd3, 0x04, 0x5f, 0x1b, 0xd9, 0xe9, 0x44, 0xfe, 0x54, 0xb7, 0x43, 0xbe, 0x12, - 0x6f, 0x1d, 0xd7, 0xd7, 0xb9, 0xb6, 0x7a, 0x8e, 0x79, 0xd9, 0x10, 0x4e, 0x84, 0x14, 0xbd, 0x0f, - 0x63, 0xf5, 0x48, 0x59, 0x73, 0x70, 0x5d, 0x73, 0x30, 0xf5, 0x2c, 0xc3, 0x6f, 0x44, 0xd5, 0xe9, - 0xbc, 0x05, 0xf0, 0x3a, 0x57, 0x57, 0x47, 0xeb, 0xf1, 0x2d, 0xb9, 0x50, 0xfe, 0x4b, 0x82, 0xa9, - 0xf4, 0xf0, 0x44, 0x31, 0xaa, 0x70, 0xda, 0xc3, 0x7e, 0x60, 0x53, 0x5f, 0x94, 0xe2, 0xfe, 0x51, - 0x7b, 0x26, 0x78, 0x09, 0x15, 0x96, 0x5c, 0xf3, 0x29, 0xb1, 0x03, 0x07, 0x6f, 0x60, 0x2f, 0x2c, - 0x9d, 0x28, 0x5b, 0xe4, 0xbd, 0xa0, 0xc3, 0x68, 0x82, 0x16, 0x9a, 0x82, 0x33, 0x0d, 0x32, 0x68, - 0x0d, 0xfe, 0x43, 0x54, 0xec, 0x07, 0x26, 0x1a, 0x86, 0xac, 0x83, 0xeb, 0x2c, 0x23, 0x3d, 0x6a, - 0xf8, 0x88, 0xce, 0x43, 0x5f, 0x9d, 0x39, 0xc9, 0x67, 0xa7, 0xa4, 0x52, 0xaf, 0x2a, 0xde, 0xe4, - 0x59, 0x28, 0x31, 0xd2, 0xfd, 0x8f, 0xb5, 0xa8, 0x2d, 0x0b, 0x7b, 0x0f, 0xc3, 0x06, 0xb5, 0xc2, - 0x5a, 0x46, 0xe0, 0xc5, 0xeb, 0x2a, 0x7f, 0x29, 0xc1, 0xcc, 0x31, 0x94, 0x45, 0x96, 0x5c, 0xc8, - 0xa7, 0xf5, 0x3d, 0xc1, 0x03, 0x25, 0x21, 0x6d, 0xdd, 0x5c, 0x8b, 0xf4, 0x8c, 0xe1, 0x24, 0x1d, - 0x79, 0x06, 0xae, 0x31, 0x70, 0xcb, 0x21, 0x69, 0x54, 0x9d, 0xe2, 0xf4, 0x40, 0xbe, 0x90, 0x44, - 0xd4, 0x5d, 0x75, 0x45, 0x1c, 0x7b, 0x70, 0x21, 0xe5, 0x4e, 0x10, 0x61, 0x94, 0x13, 0xc2, 0xe8, - 0xe2, 0x58, 0x44, 0xc1, 0xc9, 0xdd, 0xa6, 0x22, 0x6f, 0xc3, 0x45, 0x06, 0x6c, 0x93, 0xea, 0x14, - 0xef, 0x04, 0xf6, 0xa3, 0xf0, 0x1e, 0x88, 0xce, 0xd5, 0x22, 0xf4, 0xb3, 0x7b, 0x21, 0xaa, 0x79, - 0x6e, 0xa1, 0x90, 0xb0, 0x35, 0x33, 0x79, 0x60, 0x46, 0x5c, 0x22, 0xfc, 0x55, 0xfe, 0x5e, 0x82, - 0x42, 0x92, 0x6b, 0x11, 0xe5, 0x36, 0x9c, 0xe5, 0xbe, 0x6b, 0xb6, 0x6e, 0x60, 0x07, 0xbb, 0x54, - 0x6c, 0x31, 0x93, 0xb0, 0xc5, 0x43, 0xe2, 0x56, 0xb7, 0xb0, 0xe7, 0x30, 0x17, 0x1b, 0x91, 0x81, - 0xd8, 0x71, 0x88, 0xb4, 0x48, 0xd1, 0x24, 0xe4, 0x76, 0x2c, 0xdb, 0xd6, 0x74, 0x27, 0xec, 0xe9, - 0x8c, 0x93, 0xbd, 0x2a, 0x84, 0xa2, 0x25, 0x26, 0x41, 0x13, 0x30, 0x40, 0x3d, 0xab, 0x5a, 0xc5, - 0x1e, 0x36, 0x19, 0x3b, 0xfb, 0xd5, 0xa6, 0x40, 0xbe, 0x06, 0x57, 0x19, 0xec, 0x87, 0xb1, 0x1b, - 0x2d, 0xb1, 0xa8, 0x2f, 0x24, 0x98, 0x3e, 0x4a, 0x53, 0x04, 0xfb, 0x1c, 0x46, 0x13, 0x2e, 0x48, - 0x11, 0xf0, 0xd5, 0xa4, 0x80, 0x3b, 0x5c, 0x8a, 0x60, 0x91, 0xdd, 0xb1, 0x22, 0xbf, 0x94, 0xe0, - 0xd2, 0x26, 0xbb, 0xee, 0x58, 0x7e, 0x2a, 0x84, 0xec, 0x3d, 0xe1, 0xb7, 0x64, 0x54, 0xc8, 0xce, - 0x03, 0x9c, 0x6d, 0x3b, 0xc0, 0xeb, 0x30, 0xd4, 0xbc, 0x07, 0x35, 0xcb, 0x0c, 0xbb, 0x5b, 0xb6, - 0xb3, 0x75, 0xc6, 0xee, 0xcd, 0xf2, 0x66, 0xe3, 0xf9, 0x81, 0xa9, 0x0e, 0xfa, 0xb1, 0x37, 0x5f, - 0xd6, 0xa1, 0x98, 0x86, 0x48, 0xa4, 0xe4, 0x3f, 0x70, 0x5a, 0x5c, 0xe5, 0xa2, 0xa7, 0x4d, 0x26, - 0xa4, 0x81, 0xfb, 0xe0, 0xa6, 0x11, 0xbf, 0x84, 0x95, 0xfc, 0x4d, 0x16, 0xce, 0xc4, 0xd7, 0xd1, - 0x65, 0x38, 0xc3, 0xcf, 0xcd, 0x2e, 0xb6, 0xaa, 0xbb, 0x54, 0x74, 0xa9, 0x1c, 0x93, 0xad, 0x31, - 0x11, 0x1a, 0x87, 0x01, 0x7c, 0x80, 0x0d, 0xcd, 0x21, 0x26, 0x66, 0xc4, 0x18, 0x54, 0xfb, 0x43, - 0xc1, 0x3a, 0x31, 0x31, 0x7a, 0x02, 0xc3, 0x24, 0x42, 0x2b, 0xc6, 0x0c, 0xc6, 0x8e, 0xdc, 0x42, - 0x29, 0x15, 0x5a, 0x5b, 0x78, 0x6b, 0x19, 0xf5, 0x2c, 0x69, 0x15, 0x85, 0x37, 0x21, 0x27, 0x7a, - 0xc8, 0xc0, 0x7c, 0x6f, 0xea, 0x85, 0xd4, 0xe6, 0x70, 0xd5, 0xb2, 0xed, 0xb5, 0x8c, 0x3a, 0xc0, - 0x6c, 0xc3, 0x17, 0xb4, 0x0a, 0x39, 0xaa, 0xef, 0x61, 0x4f, 0x63, 0xa2, 0xfc, 0x29, 0xe6, 0xe9, - 0x4a, 0xaa, 0xa7, 0xad, 0x50, 0x97, 0xb9, 0x5b, 0xcb, 0xa8, 0x40, 0x1b, 0x6f, 0x48, 0x83, 0x91, - 0x58, 0xa9, 0x45, 0xa0, 0x7d, 0xcc, 0xdb, 0x5c, 0x97, 0x6a, 0x33, 0xa7, 0xcd, 0x9a, 0x37, 0x02, - 0x1e, 0xf6, 0xdb, 0x64, 0xcb, 0xc3, 0x30, 0xc4, 0xbd, 0x6a, 0x0e, 0xf6, 0x7d, 0xbd, 0x8a, 0xe5, - 0xcf, 0x24, 0x18, 0x4b, 0x4c, 0x18, 0x2a, 0x40, 0xbf, 0xef, 0xea, 0x35, 0x7f, 0x97, 0xf0, 0x82, - 0xf5, 0xab, 0x8d, 0x77, 0xb4, 0xdd, 0xa4, 0x08, 0x27, 0xe3, 0x9d, 0x56, 0x78, 0x62, 0x5c, 0x2c, - 0x77, 0x0e, 0x87, 0x8f, 0x76, 0x76, 0x56, 0x42, 0x01, 0xdf, 0xe4, 0xe9, 0x7c, 0x3b, 0x77, 0xbe, - 0x95, 0x60, 0x34, 0x21, 0xdf, 0x68, 0x11, 0xd8, 0x99, 0xe0, 0xe3, 0x83, 0x38, 0x9e, 0x13, 0x29, - 0x63, 0x0f, 0x1b, 0x0f, 0x54, 0x36, 0x25, 0xb1, 0x47, 0x74, 0x1b, 0xfa, 0x58, 0x65, 0x22, 0xb4, - 0xf9, 0xb4, 0x5e, 0x29, 0xd0, 0x08, 0xed, 0x90, 0xb7, 0xb1, 0x7e, 0xe5, 0xe7, 0xb3, 0x53, 0xd9, - 0x52, 0xaf, 0x9a, 0x6b, 0x36, 0x2c, 0x5f, 0x7e, 0xd1, 0x03, 0xc3, 0xed, 0x55, 0x45, 0x73, 0x70, - 0x8a, 0x33, 0x81, 0xe3, 0x4c, 0xdd, 0x6e, 0x2d, 0xa3, 0x72, 0x45, 0xb4, 0x0d, 0x23, 0xb1, 0xf6, - 0x21, 0x78, 0xd4, 0x93, 0xda, 0x75, 0xf9, 0x8e, 0xb1, 0x56, 0x14, 0xb9, 0x1b, 0xb6, 0xdb, 0x64, - 0xe8, 0x19, 0xa0, 0x18, 0x37, 0x35, 0x9f, 0xea, 0x34, 0xf0, 0xc5, 0xe9, 0x99, 0x39, 0x06, 0x45, - 0x37, 0x99, 0x81, 0x3a, 0x4c, 0xdb, 0x24, 0xcb, 0x83, 0x2d, 0xa4, 0x97, 0xbf, 0x93, 0xe0, 0x7c, - 0xb2, 0x6d, 0x98, 0xc6, 0x96, 0xcd, 0xc5, 0xf1, 0x27, 0x31, 0x95, 0x1b, 0x80, 0x3c, 0xec, 0xe8, - 0x96, 0x6b, 0xb9, 0x55, 0x6d, 0x3f, 0xd0, 0x5d, 0x1a, 0x38, 0xbe, 0xb8, 0x20, 0x46, 0x1a, 0x2b, - 0x8f, 0xc5, 0x02, 0xfa, 0x2f, 0x14, 0x49, 0x8d, 0x5a, 0x8e, 0xe5, 0x53, 0xcb, 0xd0, 0x6d, 0xfb, - 0x90, 0x1d, 0x61, 0x6c, 0x36, 0x4d, 0xf9, 0x68, 0x33, 0xd1, 0xaa, 0xb5, 0xca, 0x94, 0x22, 0x2f, - 0x0b, 0x5f, 0x01, 0x9c, 0x62, 0xd7, 0x04, 0xfa, 0x44, 0x82, 0xfe, 0x68, 0x60, 0x46, 0xb3, 0x09, - 0x59, 0x49, 0xf9, 0xea, 0x28, 0x94, 0xd2, 0x74, 0xdb, 0x3f, 0x3b, 0xe4, 0x99, 0x8f, 0x7f, 0xf9, - 0xf3, 0xf3, 0x9e, 0x2b, 0xe8, 0xb2, 0xd2, 0xe5, 0xbb, 0x51, 0xf9, 0xc0, 0x32, 0x3f, 0x44, 0x9f, - 0x4a, 0x90, 0x8b, 0x4d, 0xfe, 0xe9, 0x80, 0x3a, 0x3f, 0x41, 0x0a, 0xd7, 0x8f, 0x02, 0x14, 0xfb, - 0x94, 0x90, 0xff, 0xc1, 0x30, 0x15, 0xd1, 0x44, 0x37, 0x4c, 0xe8, 0x47, 0x09, 0xf2, 0x69, 0x23, - 0x2c, 0x5a, 0x78, 0xa7, 0x79, 0x97, 0x63, 0xbc, 0x79, 0x82, 0x19, 0x59, 0xbe, 0xcb, 0xb0, 0xde, - 0xba, 0x2b, 0xcd, 0xca, 0x8a, 0x92, 0xf8, 0xe1, 0xaa, 0xb9, 0xc4, 0xc4, 0x1a, 0x25, 0xfc, 0xbf, - 0x11, 0x03, 0xf9, 0xb3, 0x04, 0x13, 0xdd, 0xa6, 0x49, 0xb4, 0x98, 0x96, 0xb5, 0x63, 0xcc, 0xc2, - 0x85, 0x7f, 0x9f, 0xcc, 0x58, 0xc4, 0x35, 0xcd, 0xe2, 0x9a, 0x42, 0x45, 0xa5, 0xeb, 0x8f, 0x05, - 0xe8, 0x07, 0x09, 0xc6, 0xbb, 0x8c, 0x92, 0xe8, 0x6e, 0x1a, 0x8a, 0xa3, 0x87, 0xe0, 0xc2, 0xe2, - 0x89, 0x6c, 0x45, 0x00, 0x57, 0x59, 0x00, 0x93, 0xe8, 0x52, 0xd7, 0x5f, 0x50, 0xd0, 0x4f, 0x12, - 0x5c, 0x4c, 0x1d, 0xc7, 0xd0, 0x9d, 0x34, 0x04, 0x47, 0xcd, 0x7a, 0x85, 0x7f, 0x9d, 0xc0, 0x52, - 0x20, 0x2f, 0x33, 0xe4, 0x25, 0x34, 0xad, 0x1c, 0xeb, 0x57, 0x13, 0xe4, 0xc2, 0x60, 0xcb, 0xc4, - 0x8c, 0xfe, 0x99, 0xb6, 0x77, 0xd2, 0xcc, 0x5e, 0xb8, 0x71, 0x4c, 0x6d, 0x81, 0x2e, 0x83, 0x3e, - 0x8a, 0x3a, 0x6a, 0xfb, 0xa8, 0x86, 0xe6, 0x8e, 0x3b, 0xf6, 0x44, 0x73, 0x66, 0x61, 0xfe, 0x1d, - 0x2c, 0x38, 0x80, 0x39, 0x69, 0x79, 0xe3, 0xd5, 0x9b, 0xa2, 0xf4, 0xfa, 0x4d, 0x51, 0xfa, 0xe3, - 0x4d, 0x51, 0x7a, 0xf9, 0xb6, 0x98, 0x79, 0xfd, 0xb6, 0x98, 0xf9, 0xf5, 0x6d, 0x31, 0xf3, 0xde, - 0xed, 0xaa, 0x45, 0x77, 0x83, 0x4a, 0xd9, 0x20, 0x4e, 0x6b, 0xf2, 0xea, 0xb7, 0x6e, 0xb0, 0x0b, - 0x5f, 0x69, 0x48, 0x0e, 0x78, 0x42, 0xe9, 0x61, 0x0d, 0xfb, 0x95, 0x3e, 0x26, 0xbe, 0xf9, 0x77, - 0x00, 0x00, 0x00, 0xff, 0xff, 0xa1, 0x46, 0x6b, 0x53, 0x00, 0x14, 0x00, 0x00, + // 1705 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0x4f, 0x6f, 0xdc, 0xc6, + 0x15, 0x5f, 0xee, 0xca, 0xb6, 0xfc, 0xd6, 0x72, 0xe4, 0x71, 0xec, 0x6c, 0xd6, 0xf2, 0x4a, 0x66, + 0x6a, 0x7b, 0xe5, 0xd4, 0x4b, 0x59, 0x09, 0x82, 0xd4, 0x2a, 0x52, 0x48, 0x6a, 0x64, 0x19, 0xb0, + 0x1a, 0x85, 0x52, 0x1c, 0xa1, 0x0d, 0x40, 0xcc, 0x92, 0x23, 0x6a, 0x20, 0x92, 0xb3, 0x22, 0x87, + 0x0b, 0xa9, 0x45, 0x51, 0xa0, 0x87, 0x5c, 0xda, 0x02, 0x01, 0x7a, 0xe8, 0xa1, 0x40, 0x2f, 0x3d, + 0xf5, 0x50, 0xa0, 0x97, 0x1e, 0x8b, 0xb6, 0xb7, 0x5c, 0x0a, 0x18, 0xe8, 0xa5, 0x87, 0xa2, 0x28, + 0xec, 0x9e, 0xfb, 0x19, 0x0a, 0xce, 0x0c, 0x77, 0xc9, 0x5d, 0x72, 0x25, 0x0b, 0xbd, 0x48, 0x9c, + 0x37, 0xef, 0xbd, 0xf9, 0xbd, 0x37, 0xef, 0xdf, 0x2c, 0xdc, 0x76, 0x4e, 0x9c, 0xe3, 0x5e, 0xc8, + 0x38, 0xb3, 0x99, 0x67, 0xd8, 0x1e, 0xeb, 0x1a, 0x47, 0x31, 0x09, 0x4f, 0x3a, 0x82, 0x86, 0xae, + 0x65, 0xb7, 0x3b, 0xc9, 0x76, 0xf3, 0x4d, 0x97, 0xb9, 0x4c, 0x90, 0x8c, 0xe4, 0x4b, 0x32, 0x36, + 0xe7, 0x5c, 0xc6, 0x5c, 0x8f, 0x18, 0xb8, 0x47, 0x0d, 0x1c, 0x04, 0x8c, 0x63, 0x4e, 0x59, 0x10, + 0xa9, 0xdd, 0x07, 0x36, 0x8b, 0x7c, 0x16, 0x19, 0x5d, 0x1c, 0x11, 0xa9, 0xdf, 0xe8, 0x3f, 0xea, + 0x12, 0x8e, 0x1f, 0x19, 0x3d, 0xec, 0xd2, 0x40, 0x30, 0x2b, 0x5e, 0x63, 0x1c, 0x51, 0xd7, 0x63, + 0xf6, 0xa1, 0x15, 0x62, 0x4e, 0x2c, 0x8f, 0xfa, 0x94, 0x5b, 0x36, 0x0b, 0xf6, 0xa9, 0xab, 0x04, + 0xee, 0x8c, 0x0b, 0x24, 0x7f, 0xac, 0x1e, 0xa6, 0xa1, 0x62, 0x59, 0x1a, 0x67, 0x21, 0x47, 0x31, + 0xe5, 0x27, 0x16, 0xa7, 0x24, 0x2c, 0x52, 0x5a, 0xe0, 0x17, 0x16, 0x3a, 0x24, 0x55, 0x38, 0x3f, + 0xbe, 0xed, 0x63, 0x6e, 0x1f, 0x90, 0xd4, 0xe2, 0x77, 0xc7, 0x19, 0x3c, 0x7a, 0x14, 0x53, 0x47, + 0xfa, 0x25, 0x7f, 0xd8, 0xad, 0x02, 0x6d, 0xa4, 0xaf, 0x36, 0x3f, 0xca, 0x6d, 0xd2, 0xc0, 0x21, + 0xc7, 0x24, 0x34, 0xd8, 0xfe, 0xbe, 0x65, 0x1f, 0x60, 0x1a, 0x58, 0x71, 0xcf, 0xc1, 0x9c, 0x44, + 0xe3, 0x14, 0x25, 0xdf, 0xce, 0xc9, 0x47, 0x71, 0x17, 0xdb, 0x36, 0x8b, 0x03, 0x1e, 0x19, 0x11, + 0x0f, 0x09, 0xf6, 0x69, 0x90, 0xc2, 0x58, 0x2c, 0xe7, 0x1c, 0x7c, 0x4b, 0x56, 0x7d, 0x11, 0xde, + 0xfa, 0x34, 0xb9, 0xc6, 0x27, 0x84, 0xaf, 0x7b, 0xac, 0xbb, 0x8d, 0x69, 0x68, 0x92, 0xa3, 0x98, + 0x44, 0x1c, 0x5d, 0x85, 0x2a, 0x75, 0x1a, 0xda, 0x82, 0xd6, 0x9e, 0x31, 0xab, 0xd4, 0xd1, 0x3f, + 0x87, 0x1b, 0x82, 0x75, 0xc8, 0x17, 0xf5, 0x58, 0x10, 0x11, 0xf4, 0x11, 0x5c, 0x1e, 0xdc, 0x93, + 0xe0, 0xaf, 0x2f, 0xdf, 0xea, 0x8c, 0xc5, 0x5b, 0x27, 0x95, 0x5b, 0x9b, 0xfa, 0xfa, 0x5f, 0xf3, + 0x15, 0x73, 0xda, 0x56, 0x6b, 0x1d, 0x2b, 0x0c, 0xab, 0x9e, 0x37, 0x8a, 0x61, 0x03, 0x60, 0x18, + 0x57, 0x4a, 0xf7, 0xbd, 0x8e, 0x0c, 0xc2, 0x4e, 0x12, 0x84, 0x1d, 0x19, 0xe4, 0x2a, 0x08, 0x3b, + 0xdb, 0xd8, 0x25, 0x4a, 0xd6, 0xcc, 0x48, 0xea, 0xbf, 0xd5, 0xa0, 0x91, 0x03, 0xbf, 0xea, 0x79, + 0x65, 0xf8, 0x6b, 0xaf, 0x89, 0x1f, 0x3d, 0xc9, 0x81, 0xac, 0x0a, 0x90, 0xf7, 0x4f, 0x05, 0x29, + 0x0f, 0xcf, 0xa1, 0xfc, 0xa7, 0x06, 0xf3, 0x5b, 0xa4, 0xff, 0x3d, 0xe6, 0x90, 0x5d, 0x96, 0xfc, + 0x5d, 0xc7, 0x9e, 0x1d, 0x7b, 0x62, 0x33, 0xf5, 0xc8, 0x17, 0x70, 0x53, 0x66, 0x51, 0x2f, 0x64, + 0x3d, 0x16, 0x91, 0xd0, 0x52, 0xf1, 0x3a, 0xf0, 0xce, 0x38, 0xf2, 0xe7, 0xd8, 0x4b, 0xe2, 0x95, + 0x85, 0x5b, 0xa4, 0xbf, 0x25, 0xb9, 0xcd, 0x37, 0x85, 0x96, 0x6d, 0xa5, 0x44, 0x51, 0xd1, 0x0f, + 0xe0, 0x46, 0x3f, 0x65, 0xb6, 0x7c, 0xd2, 0xb7, 0x7c, 0xc2, 0x43, 0x6a, 0x47, 0x03, 0xab, 0xc6, + 0x95, 0xe7, 0x00, 0x6f, 0x49, 0x76, 0xf3, 0x7a, 0x3f, 0x7b, 0xa4, 0x24, 0xea, 0xff, 0xd5, 0x60, + 0xa1, 0xdc, 0x3c, 0x75, 0x19, 0x2e, 0x5c, 0x0a, 0x49, 0x14, 0x7b, 0x3c, 0x52, 0x57, 0xf1, 0xe4, + 0xb4, 0x33, 0x0b, 0xb4, 0x24, 0x0c, 0xab, 0x81, 0xf3, 0x9c, 0x79, 0xb1, 0x4f, 0xb6, 0x49, 0x98, + 0x5c, 0x9d, 0xba, 0xb6, 0x54, 0x7b, 0x13, 0xc3, 0xf5, 0x02, 0x2e, 0xb4, 0x00, 0x57, 0x06, 0xc1, + 0x60, 0x0d, 0xe2, 0x1f, 0xd2, 0xcb, 0x7e, 0xea, 0xa0, 0x59, 0xa8, 0xf9, 0xa4, 0x2f, 0x3c, 0x52, + 0x35, 0x93, 0x4f, 0x74, 0x13, 0x2e, 0xf6, 0x85, 0x92, 0x46, 0x6d, 0x41, 0x6b, 0x4f, 0x99, 0x6a, + 0xa5, 0x3f, 0x80, 0xb6, 0x08, 0xba, 0x8f, 0x45, 0x89, 0xda, 0xa5, 0x24, 0x7c, 0x96, 0x14, 0xa8, + 0x75, 0x51, 0x32, 0xe2, 0x30, 0x7b, 0xaf, 0xfa, 0xaf, 0x35, 0x58, 0x3c, 0x03, 0xb3, 0xf2, 0x52, + 0x00, 0x8d, 0xb2, 0xba, 0xa7, 0xe2, 0xc0, 0x28, 0x70, 0xdb, 0x24, 0xd5, 0xca, 0x3d, 0x37, 0x48, + 0x11, 0x8f, 0xbe, 0x08, 0xf7, 0x05, 0xb8, 0xb5, 0x24, 0x68, 0x4c, 0xcc, 0x49, 0xb9, 0x21, 0xbf, + 0xd2, 0x94, 0xd5, 0x13, 0x79, 0x95, 0x1d, 0x87, 0xf0, 0x56, 0x49, 0x4f, 0x50, 0x66, 0x74, 0x0a, + 0xcc, 0x98, 0xa0, 0x58, 0x59, 0x21, 0x83, 0x7b, 0x84, 0x45, 0xdf, 0x83, 0xb7, 0x05, 0xb0, 0x1d, + 0x8e, 0x39, 0xd9, 0x8f, 0xbd, 0x4f, 0x92, 0x3e, 0x90, 0xe6, 0xd5, 0x0a, 0x4c, 0x8b, 0xbe, 0x90, + 0xde, 0x79, 0x7d, 0xb9, 0x59, 0x70, 0xb4, 0x10, 0x79, 0xea, 0xa4, 0xb1, 0xc4, 0xe4, 0x52, 0xff, + 0xa3, 0x06, 0xcd, 0x22, 0xd5, 0xca, 0xca, 0x3d, 0x78, 0x43, 0xea, 0xee, 0x79, 0xd8, 0x26, 0x3e, + 0x09, 0xb8, 0x3a, 0x62, 0xb1, 0xe0, 0x88, 0x67, 0x2c, 0x70, 0x77, 0x49, 0xe8, 0x0b, 0x15, 0xdb, + 0xa9, 0x80, 0x3a, 0xf1, 0x2a, 0xcb, 0x51, 0xd1, 0x3c, 0xd4, 0xf7, 0xa9, 0xe7, 0x59, 0xd8, 0x4f, + 0x6a, 0xba, 0x88, 0xc9, 0x29, 0x13, 0x12, 0xd2, 0xaa, 0xa0, 0xa0, 0x39, 0xb8, 0xcc, 0x43, 0xea, + 0xba, 0x24, 0x24, 0x8e, 0x88, 0xce, 0x69, 0x73, 0x48, 0xd0, 0xef, 0xc3, 0x5d, 0x01, 0xfb, 0x59, + 0xa6, 0xa3, 0x15, 0x5e, 0xea, 0x97, 0x1a, 0xdc, 0x3b, 0x8d, 0x53, 0x19, 0xfb, 0x05, 0x5c, 0x2f, + 0x68, 0x90, 0xca, 0xe0, 0xbb, 0x45, 0x06, 0x8f, 0xa9, 0x54, 0xc6, 0x22, 0x6f, 0x6c, 0x47, 0xff, + 0x4a, 0x83, 0xdb, 0x3b, 0xa2, 0xdd, 0x09, 0xff, 0x74, 0x19, 0x3b, 0xfc, 0x4c, 0x76, 0xc9, 0xf4, + 0x22, 0xc7, 0x13, 0xb8, 0x36, 0x92, 0xc0, 0x5b, 0x70, 0x75, 0xd8, 0x07, 0x2d, 0xea, 0x24, 0xd5, + 0xad, 0x36, 0x5e, 0x3a, 0x33, 0x7d, 0xb3, 0xb3, 0x33, 0xf8, 0x7e, 0xea, 0x98, 0x33, 0x51, 0x66, + 0x15, 0xe9, 0x18, 0x5a, 0x65, 0x88, 0x94, 0x4b, 0xbe, 0x03, 0x97, 0x54, 0x2b, 0x57, 0x35, 0x6d, + 0xbe, 0xc0, 0x0d, 0x52, 0x87, 0x14, 0x4d, 0xe3, 0x4b, 0x49, 0xe9, 0xbf, 0xab, 0xc1, 0x95, 0xec, + 0x3e, 0xba, 0x03, 0x57, 0x64, 0xde, 0x1c, 0x10, 0xea, 0x1e, 0x70, 0x55, 0xa5, 0xea, 0x82, 0xb6, + 0x29, 0x48, 0xe8, 0x16, 0x5c, 0x26, 0xc7, 0xc4, 0xb6, 0x7c, 0xe6, 0x10, 0x11, 0x18, 0x33, 0xe6, + 0x74, 0x42, 0xd8, 0x62, 0x0e, 0x41, 0x9f, 0xc1, 0x2c, 0x4b, 0xd1, 0xaa, 0x31, 0x43, 0x44, 0x47, + 0x7d, 0xb9, 0x5d, 0x0a, 0x6d, 0xc4, 0xbc, 0xcd, 0x8a, 0xf9, 0x06, 0xcb, 0x93, 0x92, 0x4e, 0x28, + 0x03, 0x3d, 0x89, 0xc0, 0xc6, 0x54, 0x69, 0x43, 0x1a, 0x51, 0xb8, 0x41, 0x3d, 0x6f, 0xb3, 0x62, + 0x5e, 0x16, 0xb2, 0xc9, 0x02, 0x6d, 0x40, 0x9d, 0xe3, 0x43, 0x12, 0x5a, 0x82, 0xd4, 0xb8, 0x20, + 0x34, 0xbd, 0x53, 0xaa, 0x69, 0x37, 0xe1, 0x15, 0xea, 0x36, 0x2b, 0x26, 0xf0, 0xc1, 0x0a, 0x59, + 0x70, 0x2d, 0x73, 0xd5, 0xca, 0xd0, 0x8b, 0x42, 0xdb, 0xd2, 0x84, 0xdb, 0x16, 0x4a, 0x87, 0x77, + 0x3e, 0x30, 0x78, 0x36, 0x1a, 0xa1, 0xad, 0xcd, 0xc2, 0x55, 0xa9, 0xd5, 0xf2, 0x49, 0x14, 0x61, + 0x97, 0xe8, 0x7f, 0xd3, 0xa0, 0xb1, 0xc3, 0xb1, 0x4b, 0x9c, 0x0d, 0x1a, 0x60, 0x8f, 0xfe, 0x90, + 0x88, 0x7a, 0xf5, 0x71, 0x3f, 0xc9, 0xd7, 0xbc, 0x83, 0xb4, 0xf3, 0x3b, 0xa8, 0xd0, 0xb0, 0xea, + 0xff, 0xd1, 0xb0, 0x4b, 0x70, 0x81, 0x24, 0x90, 0xf5, 0x5f, 0x68, 0x70, 0xa3, 0x30, 0x00, 0x50, + 0x13, 0xa6, 0xa3, 0x00, 0xf7, 0xa2, 0x03, 0x26, 0x03, 0x70, 0xda, 0x1c, 0xac, 0xd1, 0xde, 0x30, + 0xe4, 0x65, 0x72, 0x7d, 0x98, 0x47, 0xa5, 0xc6, 0xdf, 0xce, 0xf8, 0xb0, 0xfb, 0xc9, 0xfe, 0xfe, + 0x7a, 0x42, 0x90, 0x87, 0x3c, 0x7f, 0x34, 0x9a, 0x0b, 0xbf, 0xd7, 0xe0, 0x7a, 0x81, 0x7b, 0xd0, + 0x0a, 0x88, 0x1c, 0x97, 0xe3, 0x90, 0x72, 0xed, 0x5c, 0xc9, 0x18, 0x27, 0xc6, 0x1d, 0x53, 0x4c, + 0x7d, 0xe2, 0x13, 0x7d, 0x00, 0x17, 0x85, 0x6f, 0x53, 0xb4, 0x8d, 0xb2, 0xda, 0xaf, 0xd0, 0x28, + 0xee, 0x24, 0x0f, 0x33, 0xf5, 0x37, 0x6a, 0xd4, 0x16, 0x6a, 0xed, 0x29, 0xb3, 0x3e, 0x2c, 0xc0, + 0x91, 0xfe, 0x65, 0x15, 0x66, 0x47, 0xa3, 0x14, 0x2d, 0xc1, 0x05, 0x19, 0xd9, 0x12, 0x67, 0xe9, + 0x71, 0x9b, 0x15, 0x53, 0x32, 0xa2, 0x3d, 0xb8, 0x96, 0x29, 0x87, 0x2a, 0x2f, 0xaa, 0xa5, 0x5d, + 0x44, 0x9e, 0x98, 0x29, 0xad, 0xa9, 0xba, 0x59, 0x6f, 0x84, 0x86, 0x3e, 0x07, 0x94, 0xc9, 0x35, + 0x2b, 0xe2, 0x98, 0xc7, 0x91, 0xaa, 0x06, 0x8b, 0x67, 0x48, 0xb9, 0x1d, 0x21, 0x60, 0xce, 0xf2, + 0x11, 0xca, 0xda, 0x4c, 0x2e, 0x89, 0xf5, 0x3f, 0x68, 0x70, 0xb3, 0x58, 0x36, 0x71, 0x63, 0xee, + 0x70, 0x55, 0xce, 0x58, 0x86, 0xe5, 0x21, 0xa0, 0x90, 0xf8, 0x98, 0x06, 0x34, 0x70, 0xad, 0xa3, + 0x18, 0x07, 0x3c, 0xf6, 0x23, 0xd5, 0xf0, 0xae, 0x0d, 0x76, 0x3e, 0x55, 0x1b, 0xe8, 0xbb, 0xd0, + 0x62, 0x3d, 0x4e, 0x7d, 0x1a, 0x71, 0x6a, 0x63, 0xcf, 0x3b, 0x11, 0x19, 0x47, 0x9c, 0xa1, 0xa8, + 0x1c, 0xd5, 0xe6, 0xf2, 0x5c, 0x1b, 0x82, 0x29, 0xd5, 0xb2, 0xfc, 0x1b, 0x80, 0x0b, 0xa2, 0xed, + 0xa1, 0x9f, 0x69, 0x30, 0x9d, 0x3e, 0x00, 0xd0, 0x83, 0x02, 0xaf, 0x94, 0xbc, 0xa2, 0x9a, 0xed, + 0x32, 0xde, 0xd1, 0x67, 0x94, 0xbe, 0xf8, 0xd3, 0xbf, 0xff, 0xe7, 0x97, 0xd5, 0x77, 0xd0, 0x1d, + 0x63, 0xc2, 0x3b, 0xd8, 0xf8, 0x11, 0x75, 0x7e, 0x8c, 0x7e, 0xae, 0x41, 0x3d, 0xf3, 0x92, 0x29, + 0x07, 0x34, 0xfe, 0xa4, 0x6a, 0xbe, 0x7b, 0x1a, 0xa0, 0xcc, 0xd3, 0x48, 0xff, 0x86, 0xc0, 0xd4, + 0x42, 0x73, 0x93, 0x30, 0xa1, 0x3f, 0x6b, 0xd0, 0x28, 0x1b, 0xc9, 0xd1, 0xf2, 0x6b, 0xcd, 0xef, + 0x12, 0xe3, 0x7b, 0xe7, 0x98, 0xf9, 0xf5, 0xc7, 0x02, 0xeb, 0xfb, 0x8f, 0xb5, 0x07, 0xba, 0x61, + 0x14, 0x3e, 0xc4, 0xad, 0x80, 0x39, 0xc4, 0xe2, 0x4c, 0xfe, 0xb7, 0x33, 0x20, 0xff, 0xaa, 0xc1, + 0xdc, 0xa4, 0xe9, 0x18, 0xad, 0x94, 0x79, 0xed, 0x0c, 0xb3, 0x7d, 0xf3, 0xdb, 0xe7, 0x13, 0x56, + 0x76, 0xdd, 0x13, 0x76, 0x2d, 0xa0, 0x96, 0x31, 0xf1, 0xc7, 0x0f, 0xf4, 0x27, 0x0d, 0x6e, 0x4d, + 0x18, 0x8d, 0xd1, 0xe3, 0x32, 0x14, 0xa7, 0x0f, 0xf5, 0xcd, 0x95, 0x73, 0xc9, 0x2a, 0x03, 0xee, + 0x0a, 0x03, 0xe6, 0xd1, 0xed, 0x89, 0xbf, 0x08, 0xa1, 0xbf, 0x68, 0xf0, 0x76, 0xe9, 0x78, 0x89, + 0x3e, 0x2c, 0x43, 0x70, 0xda, 0xec, 0xda, 0xfc, 0xd6, 0x39, 0x24, 0x15, 0xf2, 0x8e, 0x40, 0xde, + 0x46, 0xf7, 0x8c, 0x33, 0xfd, 0x0a, 0x84, 0x02, 0x98, 0xc9, 0xbd, 0x00, 0xd0, 0x37, 0xcb, 0xce, + 0x2e, 0x7a, 0x83, 0x34, 0x1f, 0x9e, 0x91, 0x5b, 0xa1, 0xab, 0xa0, 0x9f, 0xa4, 0x15, 0x75, 0x74, + 0xf4, 0x44, 0x4b, 0x67, 0x1d, 0xe3, 0xd2, 0xb9, 0xb9, 0xf9, 0xe8, 0x35, 0x24, 0x24, 0x80, 0x25, + 0x6d, 0x6d, 0xfb, 0xeb, 0x97, 0x2d, 0xed, 0xc5, 0xcb, 0x96, 0xf6, 0xef, 0x97, 0x2d, 0xed, 0xab, + 0x57, 0xad, 0xca, 0x8b, 0x57, 0xad, 0xca, 0x3f, 0x5e, 0xb5, 0x2a, 0xdf, 0xff, 0xc0, 0xa5, 0xfc, + 0x20, 0xee, 0x76, 0x6c, 0xe6, 0xe7, 0x9d, 0xd7, 0x7f, 0xff, 0xa1, 0x68, 0xf8, 0xc6, 0x80, 0x72, + 0x2c, 0x1d, 0xca, 0x4f, 0x7a, 0x24, 0xea, 0x5e, 0x14, 0xe4, 0xf7, 0xfe, 0x17, 0x00, 0x00, 0xff, + 0xff, 0x8b, 0xcc, 0x03, 0x3a, 0xd0, 0x14, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -2553,6 +2646,80 @@ func (m *StreamUpdate_SubaccountUpdate) MarshalToSizedBuffer(dAtA []byte) (int, } return len(dAtA) - i, nil } +func (m *StagedFinalizeBlockEvent) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StagedFinalizeBlockEvent) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StagedFinalizeBlockEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Event != nil { + { + size := m.Event.Size() + i -= size + if _, err := m.Event.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *StagedFinalizeBlockEvent_OrderFill) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StagedFinalizeBlockEvent_OrderFill) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.OrderFill != nil { + { + size, err := m.OrderFill.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *StagedFinalizeBlockEvent_SubaccountUpdate) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StagedFinalizeBlockEvent_SubaccountUpdate) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.SubaccountUpdate != nil { + { + size, err := m.SubaccountUpdate.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintQuery(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} func (m *StreamOrderbookUpdate) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2621,20 +2788,20 @@ func (m *StreamOrderbookFill) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.FillAmounts) > 0 { - dAtA18 := make([]byte, len(m.FillAmounts)*10) - var j17 int + dAtA20 := make([]byte, len(m.FillAmounts)*10) + var j19 int for _, num := range m.FillAmounts { for num >= 1<<7 { - dAtA18[j17] = uint8(uint64(num)&0x7f | 0x80) + dAtA20[j19] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j17++ + j19++ } - dAtA18[j17] = uint8(num) - j17++ + dAtA20[j19] = uint8(num) + j19++ } - i -= j17 - copy(dAtA[i:], dAtA18[:j17]) - i = encodeVarintQuery(dAtA, i, uint64(j17)) + i -= j19 + copy(dAtA[i:], dAtA20[:j19]) + i = encodeVarintQuery(dAtA, i, uint64(j19)) i-- dAtA[i] = 0x1a } @@ -3098,6 +3265,42 @@ func (m *StreamUpdate_SubaccountUpdate) Size() (n int) { } return n } +func (m *StagedFinalizeBlockEvent) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Event != nil { + n += m.Event.Size() + } + return n +} + +func (m *StagedFinalizeBlockEvent_OrderFill) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.OrderFill != nil { + l = m.OrderFill.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} +func (m *StagedFinalizeBlockEvent_SubaccountUpdate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SubaccountUpdate != nil { + l = m.SubaccountUpdate.Size() + n += 1 + l + sovQuery(uint64(l)) + } + return n +} func (m *StreamOrderbookUpdate) Size() (n int) { if m == nil { return 0 @@ -4945,6 +5148,126 @@ func (m *StreamUpdate) Unmarshal(dAtA []byte) error { } return nil } +func (m *StagedFinalizeBlockEvent) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StagedFinalizeBlockEvent: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StagedFinalizeBlockEvent: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OrderFill", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &StreamOrderbookFill{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Event = &StagedFinalizeBlockEvent_OrderFill{v} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SubaccountUpdate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowQuery + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthQuery + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthQuery + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &types.StreamSubaccountUpdate{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Event = &StagedFinalizeBlockEvent_SubaccountUpdate{v} + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipQuery(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthQuery + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *StreamOrderbookUpdate) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/protocol/x/subaccounts/keeper/subaccount.go b/protocol/x/subaccounts/keeper/subaccount.go index 8b40d382b1..8ee77a9d13 100644 --- a/protocol/x/subaccounts/keeper/subaccount.go +++ b/protocol/x/subaccounts/keeper/subaccount.go @@ -445,11 +445,9 @@ func (k Keeper) UpdateSubaccounts( if lib.IsDeliverTxMode(ctx) && k.GetFullNodeStreamingManager().Enabled() { if k.GetFullNodeStreamingManager().TracksSubaccountId(*u.SettledSubaccount.Id) { subaccountUpdate := GenerateStreamSubaccountUpdate(u, fundingPayments) - k.SendFinalizedSubaccountUpdates( + k.GetFullNodeStreamingManager().StageFinalizeBlockSubaccountUpdate( ctx, - []types.StreamSubaccountUpdate{ - subaccountUpdate, - }, + subaccountUpdate, ) } } From f8c374b411eb1cf710a1cf908c5332bc92f8f983 Mon Sep 17 00:00:00 2001 From: Teddy Ding Date: Mon, 16 Sep 2024 11:40:08 -0400 Subject: [PATCH 2/5] coments --- .../src/codegen/dydxprotocol/bundle.ts | 568 +++++++++--------- .../src/codegen/dydxprotocol/clob/query.ts | 67 --- .../v4-protos/src/codegen/gogoproto/bundle.ts | 4 +- .../v4-protos/src/codegen/google/bundle.ts | 24 +- proto/dydxprotocol/clob/query.proto | 9 - proto/dydxprotocol/clob/streaming.proto | 16 + protocol/lib/metrics/metric_keys.go | 32 +- protocol/streaming/constants.go | 1 + .../streaming/full_node_streaming_manager.go | 23 +- .../clob/keeper/grpc_stream_finalize_block.go | 1 + protocol/x/clob/types/query.pb.go | 559 ++++------------- protocol/x/clob/types/streaming.pb.go | 473 +++++++++++++++ 12 files changed, 929 insertions(+), 848 deletions(-) create mode 100644 proto/dydxprotocol/clob/streaming.proto create mode 100644 protocol/x/clob/types/streaming.pb.go diff --git a/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts b/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts index 8a5e295af1..b12e68826f 100644 --- a/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts +++ b/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts @@ -1,185 +1,178 @@ import * as _5 from "./accountplus/accountplus"; import * as _6 from "./accountplus/genesis"; -import * as _7 from "./accountplus/models"; -import * as _8 from "./accountplus/params"; -import * as _9 from "./accountplus/query"; -import * as _10 from "./accountplus/tx"; -import * as _11 from "./affiliates/affiliates"; -import * as _12 from "./affiliates/genesis"; -import * as _13 from "./affiliates/query"; -import * as _14 from "./affiliates/tx"; -import * as _15 from "./assets/asset"; -import * as _16 from "./assets/genesis"; -import * as _17 from "./assets/query"; -import * as _18 from "./assets/tx"; -import * as _19 from "./blocktime/blocktime"; -import * as _20 from "./blocktime/genesis"; -import * as _21 from "./blocktime/params"; -import * as _22 from "./blocktime/query"; -import * as _23 from "./blocktime/tx"; -import * as _24 from "./bridge/bridge_event_info"; -import * as _25 from "./bridge/bridge_event"; -import * as _26 from "./bridge/genesis"; -import * as _27 from "./bridge/params"; -import * as _28 from "./bridge/query"; -import * as _29 from "./bridge/tx"; -import * as _30 from "./clob/block_rate_limit_config"; -import * as _31 from "./clob/clob_pair"; -import * as _32 from "./clob/equity_tier_limit_config"; -import * as _33 from "./clob/genesis"; -import * as _34 from "./clob/liquidations_config"; -import * as _35 from "./clob/liquidations"; -import * as _36 from "./clob/matches"; -import * as _37 from "./clob/mev"; -import * as _38 from "./clob/operation"; -import * as _39 from "./clob/order_removals"; -import * as _40 from "./clob/order"; -import * as _41 from "./clob/process_proposer_matches_events"; -import * as _42 from "./clob/query"; -import * as _43 from "./clob/tx"; -import * as _44 from "./daemons/bridge/bridge"; -import * as _45 from "./daemons/liquidation/liquidation"; -import * as _46 from "./daemons/pricefeed/price_feed"; -import * as _47 from "./delaymsg/block_message_ids"; -import * as _48 from "./delaymsg/delayed_message"; -import * as _49 from "./delaymsg/genesis"; -import * as _50 from "./delaymsg/query"; -import * as _51 from "./delaymsg/tx"; -import * as _52 from "./epochs/epoch_info"; -import * as _53 from "./epochs/genesis"; -import * as _54 from "./epochs/query"; -import * as _55 from "./feetiers/genesis"; -import * as _56 from "./feetiers/params"; -import * as _57 from "./feetiers/query"; -import * as _58 from "./feetiers/tx"; -import * as _59 from "./govplus/genesis"; -import * as _60 from "./govplus/query"; -import * as _61 from "./govplus/tx"; -import * as _62 from "./indexer/events/events"; -import * as _63 from "./indexer/indexer_manager/event"; -import * as _64 from "./indexer/off_chain_updates/off_chain_updates"; -import * as _65 from "./indexer/protocol/v1/clob"; -import * as _66 from "./indexer/protocol/v1/perpetual"; -import * as _67 from "./indexer/protocol/v1/subaccount"; -import * as _68 from "./indexer/protocol/v1/vault"; -import * as _69 from "./indexer/redis/redis_order"; -import * as _70 from "./indexer/shared/removal_reason"; -import * as _71 from "./indexer/socks/messages"; -import * as _72 from "./listing/genesis"; -import * as _73 from "./listing/params"; -import * as _74 from "./listing/query"; -import * as _75 from "./listing/tx"; -import * as _76 from "./perpetuals/genesis"; -import * as _77 from "./perpetuals/params"; -import * as _78 from "./perpetuals/perpetual"; -import * as _79 from "./perpetuals/query"; -import * as _80 from "./perpetuals/tx"; -import * as _81 from "./prices/genesis"; -import * as _82 from "./prices/market_param"; -import * as _83 from "./prices/market_price"; -import * as _84 from "./prices/query"; -import * as _85 from "./prices/tx"; -import * as _86 from "./ratelimit/capacity"; -import * as _87 from "./ratelimit/genesis"; -import * as _88 from "./ratelimit/limit_params"; -import * as _89 from "./ratelimit/pending_send_packet"; -import * as _90 from "./ratelimit/query"; -import * as _91 from "./ratelimit/tx"; -import * as _92 from "./revshare/genesis"; -import * as _93 from "./revshare/params"; -import * as _94 from "./revshare/query"; -import * as _95 from "./revshare/revshare"; -import * as _96 from "./revshare/tx"; -import * as _97 from "./rewards/genesis"; -import * as _98 from "./rewards/params"; -import * as _99 from "./rewards/query"; -import * as _100 from "./rewards/reward_share"; -import * as _101 from "./rewards/tx"; -import * as _102 from "./sending/genesis"; -import * as _103 from "./sending/query"; -import * as _104 from "./sending/transfer"; -import * as _105 from "./sending/tx"; -import * as _106 from "./stats/genesis"; -import * as _107 from "./stats/params"; -import * as _108 from "./stats/query"; -import * as _109 from "./stats/stats"; -import * as _110 from "./stats/tx"; -import * as _111 from "./subaccounts/asset_position"; -import * as _112 from "./subaccounts/genesis"; -import * as _113 from "./subaccounts/perpetual_position"; -import * as _114 from "./subaccounts/query"; -import * as _115 from "./subaccounts/streaming"; -import * as _116 from "./subaccounts/subaccount"; -import * as _117 from "./vault/genesis"; -import * as _118 from "./vault/params"; -import * as _119 from "./vault/query"; -import * as _120 from "./vault/share"; -import * as _121 from "./vault/tx"; -import * as _122 from "./vault/vault"; -import * as _123 from "./vest/genesis"; -import * as _124 from "./vest/query"; -import * as _125 from "./vest/tx"; -import * as _126 from "./vest/vest_entry"; -import * as _134 from "./accountplus/query.lcd"; -import * as _135 from "./assets/query.lcd"; -import * as _136 from "./blocktime/query.lcd"; -import * as _137 from "./bridge/query.lcd"; -import * as _138 from "./clob/query.lcd"; -import * as _139 from "./delaymsg/query.lcd"; -import * as _140 from "./epochs/query.lcd"; -import * as _141 from "./feetiers/query.lcd"; -import * as _142 from "./listing/query.lcd"; -import * as _143 from "./perpetuals/query.lcd"; -import * as _144 from "./prices/query.lcd"; -import * as _145 from "./ratelimit/query.lcd"; -import * as _146 from "./revshare/query.lcd"; -import * as _147 from "./rewards/query.lcd"; -import * as _148 from "./stats/query.lcd"; -import * as _149 from "./subaccounts/query.lcd"; -import * as _150 from "./vault/query.lcd"; -import * as _151 from "./vest/query.lcd"; -import * as _152 from "./accountplus/query.rpc.Query"; -import * as _153 from "./affiliates/query.rpc.Query"; -import * as _154 from "./assets/query.rpc.Query"; -import * as _155 from "./blocktime/query.rpc.Query"; -import * as _156 from "./bridge/query.rpc.Query"; -import * as _157 from "./clob/query.rpc.Query"; -import * as _158 from "./delaymsg/query.rpc.Query"; -import * as _159 from "./epochs/query.rpc.Query"; -import * as _160 from "./feetiers/query.rpc.Query"; -import * as _161 from "./govplus/query.rpc.Query"; -import * as _162 from "./listing/query.rpc.Query"; -import * as _163 from "./perpetuals/query.rpc.Query"; -import * as _164 from "./prices/query.rpc.Query"; -import * as _165 from "./ratelimit/query.rpc.Query"; -import * as _166 from "./revshare/query.rpc.Query"; -import * as _167 from "./rewards/query.rpc.Query"; -import * as _168 from "./sending/query.rpc.Query"; -import * as _169 from "./stats/query.rpc.Query"; -import * as _170 from "./subaccounts/query.rpc.Query"; -import * as _171 from "./vault/query.rpc.Query"; -import * as _172 from "./vest/query.rpc.Query"; -import * as _173 from "./accountplus/tx.rpc.msg"; -import * as _174 from "./affiliates/tx.rpc.msg"; -import * as _175 from "./blocktime/tx.rpc.msg"; -import * as _176 from "./bridge/tx.rpc.msg"; -import * as _177 from "./clob/tx.rpc.msg"; -import * as _178 from "./delaymsg/tx.rpc.msg"; -import * as _179 from "./feetiers/tx.rpc.msg"; -import * as _180 from "./govplus/tx.rpc.msg"; -import * as _181 from "./listing/tx.rpc.msg"; -import * as _182 from "./perpetuals/tx.rpc.msg"; -import * as _183 from "./prices/tx.rpc.msg"; -import * as _184 from "./ratelimit/tx.rpc.msg"; -import * as _185 from "./revshare/tx.rpc.msg"; -import * as _186 from "./rewards/tx.rpc.msg"; -import * as _187 from "./sending/tx.rpc.msg"; -import * as _188 from "./stats/tx.rpc.msg"; -import * as _189 from "./vault/tx.rpc.msg"; -import * as _190 from "./vest/tx.rpc.msg"; -import * as _191 from "./lcd"; -import * as _192 from "./rpc.query"; -import * as _193 from "./rpc.tx"; +import * as _7 from "./affiliates/affiliates"; +import * as _8 from "./affiliates/genesis"; +import * as _9 from "./affiliates/query"; +import * as _10 from "./affiliates/tx"; +import * as _11 from "./assets/asset"; +import * as _12 from "./assets/genesis"; +import * as _13 from "./assets/query"; +import * as _14 from "./assets/tx"; +import * as _15 from "./blocktime/blocktime"; +import * as _16 from "./blocktime/genesis"; +import * as _17 from "./blocktime/params"; +import * as _18 from "./blocktime/query"; +import * as _19 from "./blocktime/tx"; +import * as _20 from "./bridge/bridge_event_info"; +import * as _21 from "./bridge/bridge_event"; +import * as _22 from "./bridge/genesis"; +import * as _23 from "./bridge/params"; +import * as _24 from "./bridge/query"; +import * as _25 from "./bridge/tx"; +import * as _26 from "./clob/block_rate_limit_config"; +import * as _27 from "./clob/clob_pair"; +import * as _28 from "./clob/equity_tier_limit_config"; +import * as _29 from "./clob/genesis"; +import * as _30 from "./clob/liquidations_config"; +import * as _31 from "./clob/liquidations"; +import * as _32 from "./clob/matches"; +import * as _33 from "./clob/mev"; +import * as _34 from "./clob/operation"; +import * as _35 from "./clob/order_removals"; +import * as _36 from "./clob/order"; +import * as _37 from "./clob/process_proposer_matches_events"; +import * as _38 from "./clob/query"; +import * as _39 from "./clob/streaming"; +import * as _40 from "./clob/tx"; +import * as _41 from "./daemons/bridge/bridge"; +import * as _42 from "./daemons/liquidation/liquidation"; +import * as _43 from "./daemons/pricefeed/price_feed"; +import * as _44 from "./delaymsg/block_message_ids"; +import * as _45 from "./delaymsg/delayed_message"; +import * as _46 from "./delaymsg/genesis"; +import * as _47 from "./delaymsg/query"; +import * as _48 from "./delaymsg/tx"; +import * as _49 from "./epochs/epoch_info"; +import * as _50 from "./epochs/genesis"; +import * as _51 from "./epochs/query"; +import * as _52 from "./feetiers/genesis"; +import * as _53 from "./feetiers/params"; +import * as _54 from "./feetiers/query"; +import * as _55 from "./feetiers/tx"; +import * as _56 from "./govplus/genesis"; +import * as _57 from "./govplus/query"; +import * as _58 from "./govplus/tx"; +import * as _59 from "./indexer/events/events"; +import * as _60 from "./indexer/indexer_manager/event"; +import * as _61 from "./indexer/off_chain_updates/off_chain_updates"; +import * as _62 from "./indexer/protocol/v1/clob"; +import * as _63 from "./indexer/protocol/v1/perpetual"; +import * as _64 from "./indexer/protocol/v1/subaccount"; +import * as _65 from "./indexer/redis/redis_order"; +import * as _66 from "./indexer/shared/removal_reason"; +import * as _67 from "./indexer/socks/messages"; +import * as _68 from "./listing/genesis"; +import * as _69 from "./listing/params"; +import * as _70 from "./listing/query"; +import * as _71 from "./listing/tx"; +import * as _72 from "./perpetuals/genesis"; +import * as _73 from "./perpetuals/params"; +import * as _74 from "./perpetuals/perpetual"; +import * as _75 from "./perpetuals/query"; +import * as _76 from "./perpetuals/tx"; +import * as _77 from "./prices/genesis"; +import * as _78 from "./prices/market_param"; +import * as _79 from "./prices/market_price"; +import * as _80 from "./prices/query"; +import * as _81 from "./prices/tx"; +import * as _82 from "./ratelimit/capacity"; +import * as _83 from "./ratelimit/genesis"; +import * as _84 from "./ratelimit/limit_params"; +import * as _85 from "./ratelimit/pending_send_packet"; +import * as _86 from "./ratelimit/query"; +import * as _87 from "./ratelimit/tx"; +import * as _88 from "./revshare/genesis"; +import * as _89 from "./revshare/params"; +import * as _90 from "./revshare/query"; +import * as _91 from "./revshare/revshare"; +import * as _92 from "./revshare/tx"; +import * as _93 from "./rewards/genesis"; +import * as _94 from "./rewards/params"; +import * as _95 from "./rewards/query"; +import * as _96 from "./rewards/reward_share"; +import * as _97 from "./rewards/tx"; +import * as _98 from "./sending/genesis"; +import * as _99 from "./sending/query"; +import * as _100 from "./sending/transfer"; +import * as _101 from "./sending/tx"; +import * as _102 from "./stats/genesis"; +import * as _103 from "./stats/params"; +import * as _104 from "./stats/query"; +import * as _105 from "./stats/stats"; +import * as _106 from "./stats/tx"; +import * as _107 from "./subaccounts/asset_position"; +import * as _108 from "./subaccounts/genesis"; +import * as _109 from "./subaccounts/perpetual_position"; +import * as _110 from "./subaccounts/query"; +import * as _111 from "./subaccounts/streaming"; +import * as _112 from "./subaccounts/subaccount"; +import * as _113 from "./vault/genesis"; +import * as _114 from "./vault/params"; +import * as _115 from "./vault/query"; +import * as _116 from "./vault/share"; +import * as _117 from "./vault/tx"; +import * as _118 from "./vault/vault"; +import * as _119 from "./vest/genesis"; +import * as _120 from "./vest/query"; +import * as _121 from "./vest/tx"; +import * as _122 from "./vest/vest_entry"; +import * as _130 from "./assets/query.lcd"; +import * as _131 from "./blocktime/query.lcd"; +import * as _132 from "./bridge/query.lcd"; +import * as _133 from "./clob/query.lcd"; +import * as _134 from "./delaymsg/query.lcd"; +import * as _135 from "./epochs/query.lcd"; +import * as _136 from "./feetiers/query.lcd"; +import * as _137 from "./listing/query.lcd"; +import * as _138 from "./perpetuals/query.lcd"; +import * as _139 from "./prices/query.lcd"; +import * as _140 from "./ratelimit/query.lcd"; +import * as _141 from "./revshare/query.lcd"; +import * as _142 from "./rewards/query.lcd"; +import * as _143 from "./stats/query.lcd"; +import * as _144 from "./subaccounts/query.lcd"; +import * as _145 from "./vault/query.lcd"; +import * as _146 from "./vest/query.lcd"; +import * as _147 from "./affiliates/query.rpc.Query"; +import * as _148 from "./assets/query.rpc.Query"; +import * as _149 from "./blocktime/query.rpc.Query"; +import * as _150 from "./bridge/query.rpc.Query"; +import * as _151 from "./clob/query.rpc.Query"; +import * as _152 from "./delaymsg/query.rpc.Query"; +import * as _153 from "./epochs/query.rpc.Query"; +import * as _154 from "./feetiers/query.rpc.Query"; +import * as _155 from "./govplus/query.rpc.Query"; +import * as _156 from "./listing/query.rpc.Query"; +import * as _157 from "./perpetuals/query.rpc.Query"; +import * as _158 from "./prices/query.rpc.Query"; +import * as _159 from "./ratelimit/query.rpc.Query"; +import * as _160 from "./revshare/query.rpc.Query"; +import * as _161 from "./rewards/query.rpc.Query"; +import * as _162 from "./sending/query.rpc.Query"; +import * as _163 from "./stats/query.rpc.Query"; +import * as _164 from "./subaccounts/query.rpc.Query"; +import * as _165 from "./vault/query.rpc.Query"; +import * as _166 from "./vest/query.rpc.Query"; +import * as _167 from "./affiliates/tx.rpc.msg"; +import * as _168 from "./blocktime/tx.rpc.msg"; +import * as _169 from "./bridge/tx.rpc.msg"; +import * as _170 from "./clob/tx.rpc.msg"; +import * as _171 from "./delaymsg/tx.rpc.msg"; +import * as _172 from "./feetiers/tx.rpc.msg"; +import * as _173 from "./govplus/tx.rpc.msg"; +import * as _174 from "./listing/tx.rpc.msg"; +import * as _175 from "./perpetuals/tx.rpc.msg"; +import * as _176 from "./prices/tx.rpc.msg"; +import * as _177 from "./ratelimit/tx.rpc.msg"; +import * as _178 from "./revshare/tx.rpc.msg"; +import * as _179 from "./rewards/tx.rpc.msg"; +import * as _180 from "./sending/tx.rpc.msg"; +import * as _181 from "./stats/tx.rpc.msg"; +import * as _182 from "./vault/tx.rpc.msg"; +import * as _183 from "./vest/tx.rpc.msg"; +import * as _184 from "./lcd"; +import * as _185 from "./rpc.query"; +import * as _186 from "./rpc.tx"; export namespace dydxprotocol { export const accountplus = { ..._5, ..._6, @@ -187,32 +180,35 @@ export namespace dydxprotocol { ..._8, ..._9, ..._10, - ..._134, - ..._152, - ..._173 + ..._147, + ..._167 }; export const affiliates = { ..._11, ..._12, ..._13, ..._14, - ..._153, - ..._174 + ..._130, + ..._148 }; export const assets = { ..._15, ..._16, ..._17, ..._18, - ..._135, - ..._154 + ..._19, + ..._131, + ..._149, + ..._168 }; export const blocktime = { ..._19, ..._20, ..._21, ..._22, ..._23, - ..._136, - ..._155, - ..._175 + ..._24, + ..._25, + ..._132, + ..._150, + ..._169 }; export const bridge = { ..._24, ..._25, @@ -235,170 +231,166 @@ export namespace dydxprotocol { ..._38, ..._39, ..._40, - ..._41, - ..._42, - ..._43, - ..._138, - ..._157, - ..._177 + ..._133, + ..._151, + ..._170 }; export namespace daemons { - export const bridge = { ..._44 + export const bridge = { ..._41 }; - export const liquidation = { ..._45 + export const liquidation = { ..._42 }; - export const pricefeed = { ..._46 + export const pricefeed = { ..._43 }; } - export const delaymsg = { ..._47, + export const delaymsg = { ..._44, + ..._45, + ..._46, + ..._47, ..._48, - ..._49, + ..._134, + ..._152, + ..._171 + }; + export const epochs = { ..._49, ..._50, ..._51, - ..._139, - ..._158, - ..._178 + ..._135, + ..._153 }; - export const epochs = { ..._52, + export const feetiers = { ..._52, ..._53, ..._54, - ..._140, - ..._159 + ..._55, + ..._136, + ..._154, + ..._172 }; - export const feetiers = { ..._55, - ..._56, + export const govplus = { ..._56, ..._57, ..._58, - ..._141, - ..._160, - ..._179 - }; - export const govplus = { ..._59, - ..._60, - ..._61, - ..._161, - ..._180 + ..._155, + ..._173 }; export namespace indexer { - export const events = { ..._62 + export const events = { ..._59 }; - export const indexer_manager = { ..._63 + export const indexer_manager = { ..._60 }; - export const off_chain_updates = { ..._64 + export const off_chain_updates = { ..._61 }; export namespace protocol { - export const v1 = { ..._65, - ..._66, - ..._67, - ..._68 + export const v1 = { ..._62, + ..._63, + ..._64 }; } - export const redis = { ..._69 + export const redis = { ..._65 }; - export const shared = { ..._70 + export const shared = { ..._66 }; - export const socks = { ..._71 + export const socks = { ..._67 }; } - export const listing = { ..._72, + export const listing = { ..._68, + ..._69, + ..._70, + ..._71, + ..._137, + ..._156, + ..._174 + }; + export const perpetuals = { ..._72, ..._73, ..._74, ..._75, - ..._142, - ..._162, - ..._181 + ..._76, + ..._138, + ..._157, + ..._175 }; - export const perpetuals = { ..._76, - ..._77, + export const prices = { ..._77, ..._78, ..._79, ..._80, - ..._143, - ..._163, - ..._182 + ..._81, + ..._139, + ..._158, + ..._176 }; - export const prices = { ..._81, - ..._82, + export const ratelimit = { ..._82, ..._83, ..._84, ..._85, - ..._144, - ..._164, - ..._183 - }; - export const ratelimit = { ..._86, + ..._86, ..._87, - ..._88, + ..._140, + ..._159, + ..._177 + }; + export const revshare = { ..._88, ..._89, ..._90, ..._91, - ..._145, - ..._165, - ..._184 + ..._92, + ..._141, + ..._160, + ..._178 }; - export const revshare = { ..._92, - ..._93, + export const rewards = { ..._93, ..._94, ..._95, ..._96, - ..._146, - ..._166, - ..._185 + ..._97, + ..._142, + ..._161, + ..._179 }; - export const rewards = { ..._97, - ..._98, + export const sending = { ..._98, ..._99, ..._100, ..._101, - ..._147, - ..._167, - ..._186 + ..._162, + ..._180 }; - export const sending = { ..._102, + export const stats = { ..._102, ..._103, ..._104, ..._105, - ..._168, - ..._187 + ..._106, + ..._143, + ..._163, + ..._181 }; - export const stats = { ..._106, - ..._107, + export const subaccounts = { ..._107, ..._108, ..._109, ..._110, - ..._148, - ..._169, - ..._188 - }; - export const subaccounts = { ..._111, + ..._111, ..._112, - ..._113, + ..._144, + ..._164 + }; + export const vault = { ..._113, ..._114, ..._115, ..._116, - ..._149, - ..._170 - }; - export const vault = { ..._117, + ..._117, ..._118, - ..._119, + ..._145, + ..._165, + ..._182 + }; + export const vest = { ..._119, ..._120, ..._121, ..._122, - ..._150, - ..._171, - ..._189 - }; - export const vest = { ..._123, - ..._124, - ..._125, - ..._126, - ..._151, - ..._172, - ..._190 + ..._146, + ..._166, + ..._183 }; - export const ClientFactory = { ..._191, - ..._192, - ..._193 + export const ClientFactory = { ..._184, + ..._185, + ..._186 }; } \ No newline at end of file diff --git a/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts b/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts index 31cae2a8a3..41ca0c872a 100644 --- a/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts +++ b/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/query.ts @@ -319,18 +319,6 @@ export interface StreamUpdateSDKType { taker_order?: StreamTakerOrderSDKType; subaccount_update?: StreamSubaccountUpdateSDKType; } -/** StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. */ - -export interface StagedFinalizeBlockEvent { - orderFill?: StreamOrderbookFill; - subaccountUpdate?: StreamSubaccountUpdate; -} -/** StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. */ - -export interface StagedFinalizeBlockEventSDKType { - order_fill?: StreamOrderbookFillSDKType; - subaccount_update?: StreamSubaccountUpdateSDKType; -} /** * StreamOrderbookUpdate provides information on an orderbook update. Used in * the full node GRPC stream. @@ -1408,61 +1396,6 @@ export const StreamUpdate = { }; -function createBaseStagedFinalizeBlockEvent(): StagedFinalizeBlockEvent { - return { - orderFill: undefined, - subaccountUpdate: undefined - }; -} - -export const StagedFinalizeBlockEvent = { - encode(message: StagedFinalizeBlockEvent, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { - if (message.orderFill !== undefined) { - StreamOrderbookFill.encode(message.orderFill, writer.uint32(10).fork()).ldelim(); - } - - if (message.subaccountUpdate !== undefined) { - StreamSubaccountUpdate.encode(message.subaccountUpdate, writer.uint32(18).fork()).ldelim(); - } - - return writer; - }, - - decode(input: _m0.Reader | Uint8Array, length?: number): StagedFinalizeBlockEvent { - const reader = input instanceof _m0.Reader ? input : new _m0.Reader(input); - let end = length === undefined ? reader.len : reader.pos + length; - const message = createBaseStagedFinalizeBlockEvent(); - - while (reader.pos < end) { - const tag = reader.uint32(); - - switch (tag >>> 3) { - case 1: - message.orderFill = StreamOrderbookFill.decode(reader, reader.uint32()); - break; - - case 2: - message.subaccountUpdate = StreamSubaccountUpdate.decode(reader, reader.uint32()); - break; - - default: - reader.skipType(tag & 7); - break; - } - } - - return message; - }, - - fromPartial(object: DeepPartial): StagedFinalizeBlockEvent { - const message = createBaseStagedFinalizeBlockEvent(); - message.orderFill = object.orderFill !== undefined && object.orderFill !== null ? StreamOrderbookFill.fromPartial(object.orderFill) : undefined; - message.subaccountUpdate = object.subaccountUpdate !== undefined && object.subaccountUpdate !== null ? StreamSubaccountUpdate.fromPartial(object.subaccountUpdate) : undefined; - return message; - } - -}; - function createBaseStreamOrderbookUpdate(): StreamOrderbookUpdate { return { snapshot: false, diff --git a/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts b/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts index 52cc244930..01ed58731a 100644 --- a/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts +++ b/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts @@ -1,3 +1,3 @@ -import * as _127 from "./gogo"; -export const gogoproto = { ..._127 +import * as _123 from "./gogo"; +export const gogoproto = { ..._123 }; \ No newline at end of file diff --git a/indexer/packages/v4-protos/src/codegen/google/bundle.ts b/indexer/packages/v4-protos/src/codegen/google/bundle.ts index fd95177ed2..a2768945ac 100644 --- a/indexer/packages/v4-protos/src/codegen/google/bundle.ts +++ b/indexer/packages/v4-protos/src/codegen/google/bundle.ts @@ -1,16 +1,16 @@ -import * as _128 from "./api/annotations"; -import * as _129 from "./api/http"; -import * as _130 from "./protobuf/descriptor"; -import * as _131 from "./protobuf/duration"; -import * as _132 from "./protobuf/timestamp"; -import * as _133 from "./protobuf/any"; +import * as _124 from "./api/annotations"; +import * as _125 from "./api/http"; +import * as _126 from "./protobuf/descriptor"; +import * as _127 from "./protobuf/duration"; +import * as _128 from "./protobuf/timestamp"; +import * as _129 from "./protobuf/any"; export namespace google { - export const api = { ..._128, - ..._129 + export const api = { ..._124, + ..._125 }; - export const protobuf = { ..._130, - ..._131, - ..._132, - ..._133 + export const protobuf = { ..._126, + ..._127, + ..._128, + ..._129 }; } \ No newline at end of file diff --git a/proto/dydxprotocol/clob/query.proto b/proto/dydxprotocol/clob/query.proto index fb4757e875..9d47ea4641 100644 --- a/proto/dydxprotocol/clob/query.proto +++ b/proto/dydxprotocol/clob/query.proto @@ -198,15 +198,6 @@ message StreamUpdate { } } -// StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. -message StagedFinalizeBlockEvent { - // Contains one of StreamOrderbookFill, StreamSubaccountUpdate. - oneof event { - StreamOrderbookFill order_fill = 1; - dydxprotocol.subaccounts.StreamSubaccountUpdate subaccount_update = 2; - } -} - // StreamOrderbookUpdate provides information on an orderbook update. Used in // the full node GRPC stream. message StreamOrderbookUpdate { diff --git a/proto/dydxprotocol/clob/streaming.proto b/proto/dydxprotocol/clob/streaming.proto new file mode 100644 index 0000000000..06c74ffbe1 --- /dev/null +++ b/proto/dydxprotocol/clob/streaming.proto @@ -0,0 +1,16 @@ +syntax = "proto3"; +package dydxprotocol.clob; + +import "dydxprotocol/subaccounts/streaming.proto"; +import "dydxprotocol/clob/query.proto"; + +option go_package = "github.com/dydxprotocol/v4-chain/protocol/x/clob/types"; + +// StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. +message StagedFinalizeBlockEvent { + // Contains one of StreamOrderbookFill, StreamSubaccountUpdate. + oneof event { + StreamOrderbookFill order_fill = 1; + dydxprotocol.subaccounts.StreamSubaccountUpdate subaccount_update = 2; + } +} diff --git a/protocol/lib/metrics/metric_keys.go b/protocol/lib/metrics/metric_keys.go index 2a44834332..c7c42b4c44 100644 --- a/protocol/lib/metrics/metric_keys.go +++ b/protocol/lib/metrics/metric_keys.go @@ -70,22 +70,22 @@ const ( GateWithdrawalsIfNegativeTncSubaccountSeenLatency = "gate_withdrawals_if_negative_tnc_subaccount_seen_latency" // Full node grpc - FullNodeGrpc = "full_node_grpc" - GrpcSendOrderbookUpdatesLatency = "grpc_send_orderbook_updates_latency" - GrpcSendOrderbookSnapshotLatency = "grpc_send_orderbook_snapshot_latency" - GrpcSendSubaccountSnapshotLatency = "grpc_send_subaccount_snapshot_latency" - GrpcSendOrderbookFillsLatency = "grpc_send_orderbook_fills_latency" - GrpcSendFinalizedSubaccountUpdatesLatency = "grpc_send_finalized_subaccount_updates_latency" - GrpcAddUpdateToBufferCount = "grpc_add_update_to_buffer_count" - GrpcAddToSubscriptionChannelCount = "grpc_add_to_subscription_channel_count" - GrpcSendResponseToSubscriberCount = "grpc_send_response_to_subscriber_count" - GrpcStreamSubscriberCount = "grpc_stream_subscriber_count" - GrpcStreamNumUpdatesBuffered = "grpc_stream_num_updates_buffered" - GrpcFlushUpdatesLatency = "grpc_flush_updates_latency" - GrpcSubscriptionChannelLength = "grpc_subscription_channel_length" - GrpcStagedAllFinalizeBlockUpdates = "grpc_staged_all_finalize_block_updates" - GrpcStagedFillFinalizeBlockUpdates = "grpc_staged_finalize_block_fill_updates" - GrpcStagedSubaccountFinalizeBlockUpdates = "grpc_staged_finalize_block_subaccount_updates" + FullNodeGrpc = "full_node_grpc" + GrpcSendOrderbookUpdatesLatency = "grpc_send_orderbook_updates_latency" + GrpcSendOrderbookSnapshotLatency = "grpc_send_orderbook_snapshot_latency" + GrpcSendSubaccountSnapshotLatency = "grpc_send_subaccount_snapshot_latency" + GrpcSendOrderbookFillsLatency = "grpc_send_orderbook_fills_latency" + GrpcSendFinalizedSubaccountUpdatesLatency = "grpc_send_finalized_subaccount_updates_latency" + GrpcAddUpdateToBufferCount = "grpc_add_update_to_buffer_count" + GrpcAddToSubscriptionChannelCount = "grpc_add_to_subscription_channel_count" + GrpcSendResponseToSubscriberCount = "grpc_send_response_to_subscriber_count" + GrpcStreamSubscriberCount = "grpc_stream_subscriber_count" + GrpcStreamNumUpdatesBuffered = "grpc_stream_num_updates_buffered" + GrpcFlushUpdatesLatency = "grpc_flush_updates_latency" + GrpcSubscriptionChannelLength = "grpc_subscription_channel_length" + GrpcStagedAllFinalizeBlockUpdatesCount = "grpc_staged_all_finalize_block_updates_count" + GrpcStagedFillFinalizeBlockUpdatesCount = "grpc_staged_finalize_block_fill_updates_count" + GrpcStagedSubaccountFinalizeBlockUpdatesCount = "grpc_staged_finalize_block_subaccount_updates_count" EndBlocker = "end_blocker" EndBlockerLag = "end_blocker_lag" diff --git a/protocol/streaming/constants.go b/protocol/streaming/constants.go index c3c3c255a7..3e3a3a6676 100644 --- a/protocol/streaming/constants.go +++ b/protocol/streaming/constants.go @@ -1,5 +1,6 @@ package streaming +// Constants for FullNodeStreamingManager. const ( // Transient store key for storing staged events. StreamingManagerTransientStoreKey = "tmp_streaming" diff --git a/protocol/streaming/full_node_streaming_manager.go b/protocol/streaming/full_node_streaming_manager.go index f357908f28..0bf1f91476 100644 --- a/protocol/streaming/full_node_streaming_manager.go +++ b/protocol/streaming/full_node_streaming_manager.go @@ -13,7 +13,6 @@ import ( "cosmossdk.io/log" "cosmossdk.io/store/prefix" storetypes "cosmossdk.io/store/types" - "github.com/cosmos/cosmos-sdk/telemetry" sdk "github.com/cosmos/cosmos-sdk/types" ante_types "github.com/dydxprotocol/v4-chain/protocol/app/ante/types" "github.com/dydxprotocol/v4-chain/protocol/lib/metrics" @@ -404,6 +403,10 @@ func (sm *FullNodeStreamingManagerImpl) StageFinalizeBlockSubaccountUpdate( } // Stage a fill event in transient store, during `FinalizeBlock`. +// Since `FinalizeBlock` code block can be called more than once with optimistc +// execution (once optimistcally and optionally once on the canonical block), +// we need to stage the events in transient store and later emit them +// during `Precommit`. func (sm *FullNodeStreamingManagerImpl) StageFinalizeBlockFill( ctx sdk.Context, fill clobtypes.StreamOrderbookFill, @@ -841,11 +844,9 @@ func (sm *FullNodeStreamingManagerImpl) getStagedEventsFromFinalizeBlock( // Get onchain stream events stored in transient store. stagedEvents := sm.GetStagedFinalizeBlockEvents(ctx) - telemetry.SetGauge( + metrics.SetGauge( + metrics.GrpcStagedAllFinalizeBlockUpdatesCount, float32(len(stagedEvents)), - types.ModuleName, - metrics.GrpcStagedAllFinalizeBlockUpdates, - metrics.Count, ) for _, stagedEvent := range stagedEvents { @@ -857,17 +858,13 @@ func (sm *FullNodeStreamingManagerImpl) getStagedEventsFromFinalizeBlock( } } - telemetry.SetGauge( + metrics.SetGauge( + metrics.GrpcStagedSubaccountFinalizeBlockUpdatesCount, float32(len(finalizedSubaccountUpdates)), - types.ModuleName, - metrics.GrpcStagedSubaccountFinalizeBlockUpdates, - metrics.Count, ) - telemetry.SetGauge( + metrics.SetGauge( + metrics.GrpcStagedFillFinalizeBlockUpdatesCount, float32(len(finalizedFills)), - types.ModuleName, - metrics.GrpcStagedFillFinalizeBlockUpdates, - metrics.Count, ) return finalizedFills, finalizedSubaccountUpdates diff --git a/protocol/x/clob/keeper/grpc_stream_finalize_block.go b/protocol/x/clob/keeper/grpc_stream_finalize_block.go index 6216eba461..1bc23e46c6 100644 --- a/protocol/x/clob/keeper/grpc_stream_finalize_block.go +++ b/protocol/x/clob/keeper/grpc_stream_finalize_block.go @@ -11,6 +11,7 @@ import ( // Returns the order updates needed to update the fill amount for the orders // from local ops queue, according to the latest onchain state (after FinalizeBlock). +// Effectively reverts the optimistic fill amounts removed from the CheckTx to DeliverTx state transition. func (k Keeper) getUpdatesToSyncLocalOpsQueue( ctx sdk.Context, ) *types.OffchainUpdates { diff --git a/protocol/x/clob/types/query.pb.go b/protocol/x/clob/types/query.pb.go index 279280a0ae..4f4d01af14 100644 --- a/protocol/x/clob/types/query.pb.go +++ b/protocol/x/clob/types/query.pb.go @@ -1004,95 +1004,6 @@ func (*StreamUpdate) XXX_OneofWrappers() []interface{} { } } -// StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. -type StagedFinalizeBlockEvent struct { - // Contains one of StreamOrderbookFill, StreamSubaccountUpdate. - // - // Types that are valid to be assigned to Event: - // - // *StagedFinalizeBlockEvent_OrderFill - // *StagedFinalizeBlockEvent_SubaccountUpdate - Event isStagedFinalizeBlockEvent_Event `protobuf_oneof:"event"` -} - -func (m *StagedFinalizeBlockEvent) Reset() { *m = StagedFinalizeBlockEvent{} } -func (m *StagedFinalizeBlockEvent) String() string { return proto.CompactTextString(m) } -func (*StagedFinalizeBlockEvent) ProtoMessage() {} -func (*StagedFinalizeBlockEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{17} -} -func (m *StagedFinalizeBlockEvent) XXX_Unmarshal(b []byte) error { - return m.Unmarshal(b) -} -func (m *StagedFinalizeBlockEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - if deterministic { - return xxx_messageInfo_StagedFinalizeBlockEvent.Marshal(b, m, deterministic) - } else { - b = b[:cap(b)] - n, err := m.MarshalToSizedBuffer(b) - if err != nil { - return nil, err - } - return b[:n], nil - } -} -func (m *StagedFinalizeBlockEvent) XXX_Merge(src proto.Message) { - xxx_messageInfo_StagedFinalizeBlockEvent.Merge(m, src) -} -func (m *StagedFinalizeBlockEvent) XXX_Size() int { - return m.Size() -} -func (m *StagedFinalizeBlockEvent) XXX_DiscardUnknown() { - xxx_messageInfo_StagedFinalizeBlockEvent.DiscardUnknown(m) -} - -var xxx_messageInfo_StagedFinalizeBlockEvent proto.InternalMessageInfo - -type isStagedFinalizeBlockEvent_Event interface { - isStagedFinalizeBlockEvent_Event() - MarshalTo([]byte) (int, error) - Size() int -} - -type StagedFinalizeBlockEvent_OrderFill struct { - OrderFill *StreamOrderbookFill `protobuf:"bytes,1,opt,name=order_fill,json=orderFill,proto3,oneof" json:"order_fill,omitempty"` -} -type StagedFinalizeBlockEvent_SubaccountUpdate struct { - SubaccountUpdate *types.StreamSubaccountUpdate `protobuf:"bytes,2,opt,name=subaccount_update,json=subaccountUpdate,proto3,oneof" json:"subaccount_update,omitempty"` -} - -func (*StagedFinalizeBlockEvent_OrderFill) isStagedFinalizeBlockEvent_Event() {} -func (*StagedFinalizeBlockEvent_SubaccountUpdate) isStagedFinalizeBlockEvent_Event() {} - -func (m *StagedFinalizeBlockEvent) GetEvent() isStagedFinalizeBlockEvent_Event { - if m != nil { - return m.Event - } - return nil -} - -func (m *StagedFinalizeBlockEvent) GetOrderFill() *StreamOrderbookFill { - if x, ok := m.GetEvent().(*StagedFinalizeBlockEvent_OrderFill); ok { - return x.OrderFill - } - return nil -} - -func (m *StagedFinalizeBlockEvent) GetSubaccountUpdate() *types.StreamSubaccountUpdate { - if x, ok := m.GetEvent().(*StagedFinalizeBlockEvent_SubaccountUpdate); ok { - return x.SubaccountUpdate - } - return nil -} - -// XXX_OneofWrappers is for the internal use of the proto package. -func (*StagedFinalizeBlockEvent) XXX_OneofWrappers() []interface{} { - return []interface{}{ - (*StagedFinalizeBlockEvent_OrderFill)(nil), - (*StagedFinalizeBlockEvent_SubaccountUpdate)(nil), - } -} - // StreamOrderbookUpdate provides information on an orderbook update. Used in // the full node GRPC stream. type StreamOrderbookUpdate struct { @@ -1110,7 +1021,7 @@ func (m *StreamOrderbookUpdate) Reset() { *m = StreamOrderbookUpdate{} } func (m *StreamOrderbookUpdate) String() string { return proto.CompactTextString(m) } func (*StreamOrderbookUpdate) ProtoMessage() {} func (*StreamOrderbookUpdate) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{18} + return fileDescriptor_3365c195b25c5bc0, []int{17} } func (m *StreamOrderbookUpdate) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1170,7 +1081,7 @@ func (m *StreamOrderbookFill) Reset() { *m = StreamOrderbookFill{} } func (m *StreamOrderbookFill) String() string { return proto.CompactTextString(m) } func (*StreamOrderbookFill) ProtoMessage() {} func (*StreamOrderbookFill) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{19} + return fileDescriptor_3365c195b25c5bc0, []int{18} } func (m *StreamOrderbookFill) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1241,7 +1152,7 @@ func (m *StreamTakerOrder) Reset() { *m = StreamTakerOrder{} } func (m *StreamTakerOrder) String() string { return proto.CompactTextString(m) } func (*StreamTakerOrder) ProtoMessage() {} func (*StreamTakerOrder) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{20} + return fileDescriptor_3365c195b25c5bc0, []int{19} } func (m *StreamTakerOrder) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1344,7 +1255,7 @@ func (m *StreamTakerOrderStatus) Reset() { *m = StreamTakerOrderStatus{} func (m *StreamTakerOrderStatus) String() string { return proto.CompactTextString(m) } func (*StreamTakerOrderStatus) ProtoMessage() {} func (*StreamTakerOrderStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_3365c195b25c5bc0, []int{21} + return fileDescriptor_3365c195b25c5bc0, []int{20} } func (m *StreamTakerOrderStatus) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1413,7 +1324,6 @@ func init() { proto.RegisterType((*StreamOrderbookUpdatesRequest)(nil), "dydxprotocol.clob.StreamOrderbookUpdatesRequest") proto.RegisterType((*StreamOrderbookUpdatesResponse)(nil), "dydxprotocol.clob.StreamOrderbookUpdatesResponse") proto.RegisterType((*StreamUpdate)(nil), "dydxprotocol.clob.StreamUpdate") - proto.RegisterType((*StagedFinalizeBlockEvent)(nil), "dydxprotocol.clob.StagedFinalizeBlockEvent") proto.RegisterType((*StreamOrderbookUpdate)(nil), "dydxprotocol.clob.StreamOrderbookUpdate") proto.RegisterType((*StreamOrderbookFill)(nil), "dydxprotocol.clob.StreamOrderbookFill") proto.RegisterType((*StreamTakerOrder)(nil), "dydxprotocol.clob.StreamTakerOrder") @@ -1423,114 +1333,111 @@ func init() { func init() { proto.RegisterFile("dydxprotocol/clob/query.proto", fileDescriptor_3365c195b25c5bc0) } var fileDescriptor_3365c195b25c5bc0 = []byte{ - // 1705 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x58, 0x4f, 0x6f, 0xdc, 0xc6, - 0x15, 0x5f, 0xee, 0xca, 0xb6, 0xfc, 0xd6, 0x72, 0xe4, 0x71, 0xec, 0x6c, 0xd6, 0xf2, 0x4a, 0x66, - 0x6a, 0x7b, 0xe5, 0xd4, 0x4b, 0x59, 0x09, 0x82, 0xd4, 0x2a, 0x52, 0x48, 0x6a, 0x64, 0x19, 0xb0, - 0x1a, 0x85, 0x52, 0x1c, 0xa1, 0x0d, 0x40, 0xcc, 0x92, 0x23, 0x6a, 0x20, 0x92, 0xb3, 0x22, 0x87, - 0x0b, 0xa9, 0x45, 0x51, 0xa0, 0x87, 0x5c, 0xda, 0x02, 0x01, 0x7a, 0xe8, 0xa1, 0x40, 0x2f, 0x3d, - 0xf5, 0x50, 0xa0, 0x97, 0x1e, 0x8b, 0xb6, 0xb7, 0x5c, 0x0a, 0x18, 0xe8, 0xa5, 0x87, 0xa2, 0x28, - 0xec, 0x9e, 0xfb, 0x19, 0x0a, 0xce, 0x0c, 0x77, 0xc9, 0x5d, 0x72, 0x25, 0x0b, 0xbd, 0x48, 0x9c, - 0x37, 0xef, 0xbd, 0xf9, 0xbd, 0x37, 0xef, 0xdf, 0x2c, 0xdc, 0x76, 0x4e, 0x9c, 0xe3, 0x5e, 0xc8, - 0x38, 0xb3, 0x99, 0x67, 0xd8, 0x1e, 0xeb, 0x1a, 0x47, 0x31, 0x09, 0x4f, 0x3a, 0x82, 0x86, 0xae, - 0x65, 0xb7, 0x3b, 0xc9, 0x76, 0xf3, 0x4d, 0x97, 0xb9, 0x4c, 0x90, 0x8c, 0xe4, 0x4b, 0x32, 0x36, - 0xe7, 0x5c, 0xc6, 0x5c, 0x8f, 0x18, 0xb8, 0x47, 0x0d, 0x1c, 0x04, 0x8c, 0x63, 0x4e, 0x59, 0x10, - 0xa9, 0xdd, 0x07, 0x36, 0x8b, 0x7c, 0x16, 0x19, 0x5d, 0x1c, 0x11, 0xa9, 0xdf, 0xe8, 0x3f, 0xea, - 0x12, 0x8e, 0x1f, 0x19, 0x3d, 0xec, 0xd2, 0x40, 0x30, 0x2b, 0x5e, 0x63, 0x1c, 0x51, 0xd7, 0x63, - 0xf6, 0xa1, 0x15, 0x62, 0x4e, 0x2c, 0x8f, 0xfa, 0x94, 0x5b, 0x36, 0x0b, 0xf6, 0xa9, 0xab, 0x04, - 0xee, 0x8c, 0x0b, 0x24, 0x7f, 0xac, 0x1e, 0xa6, 0xa1, 0x62, 0x59, 0x1a, 0x67, 0x21, 0x47, 0x31, - 0xe5, 0x27, 0x16, 0xa7, 0x24, 0x2c, 0x52, 0x5a, 0xe0, 0x17, 0x16, 0x3a, 0x24, 0x55, 0x38, 0x3f, - 0xbe, 0xed, 0x63, 0x6e, 0x1f, 0x90, 0xd4, 0xe2, 0x77, 0xc7, 0x19, 0x3c, 0x7a, 0x14, 0x53, 0x47, - 0xfa, 0x25, 0x7f, 0xd8, 0xad, 0x02, 0x6d, 0xa4, 0xaf, 0x36, 0x3f, 0xca, 0x6d, 0xd2, 0xc0, 0x21, - 0xc7, 0x24, 0x34, 0xd8, 0xfe, 0xbe, 0x65, 0x1f, 0x60, 0x1a, 0x58, 0x71, 0xcf, 0xc1, 0x9c, 0x44, - 0xe3, 0x14, 0x25, 0xdf, 0xce, 0xc9, 0x47, 0x71, 0x17, 0xdb, 0x36, 0x8b, 0x03, 0x1e, 0x19, 0x11, - 0x0f, 0x09, 0xf6, 0x69, 0x90, 0xc2, 0x58, 0x2c, 0xe7, 0x1c, 0x7c, 0x4b, 0x56, 0x7d, 0x11, 0xde, - 0xfa, 0x34, 0xb9, 0xc6, 0x27, 0x84, 0xaf, 0x7b, 0xac, 0xbb, 0x8d, 0x69, 0x68, 0x92, 0xa3, 0x98, - 0x44, 0x1c, 0x5d, 0x85, 0x2a, 0x75, 0x1a, 0xda, 0x82, 0xd6, 0x9e, 0x31, 0xab, 0xd4, 0xd1, 0x3f, - 0x87, 0x1b, 0x82, 0x75, 0xc8, 0x17, 0xf5, 0x58, 0x10, 0x11, 0xf4, 0x11, 0x5c, 0x1e, 0xdc, 0x93, - 0xe0, 0xaf, 0x2f, 0xdf, 0xea, 0x8c, 0xc5, 0x5b, 0x27, 0x95, 0x5b, 0x9b, 0xfa, 0xfa, 0x5f, 0xf3, - 0x15, 0x73, 0xda, 0x56, 0x6b, 0x1d, 0x2b, 0x0c, 0xab, 0x9e, 0x37, 0x8a, 0x61, 0x03, 0x60, 0x18, - 0x57, 0x4a, 0xf7, 0xbd, 0x8e, 0x0c, 0xc2, 0x4e, 0x12, 0x84, 0x1d, 0x19, 0xe4, 0x2a, 0x08, 0x3b, - 0xdb, 0xd8, 0x25, 0x4a, 0xd6, 0xcc, 0x48, 0xea, 0xbf, 0xd5, 0xa0, 0x91, 0x03, 0xbf, 0xea, 0x79, - 0x65, 0xf8, 0x6b, 0xaf, 0x89, 0x1f, 0x3d, 0xc9, 0x81, 0xac, 0x0a, 0x90, 0xf7, 0x4f, 0x05, 0x29, - 0x0f, 0xcf, 0xa1, 0xfc, 0xa7, 0x06, 0xf3, 0x5b, 0xa4, 0xff, 0x3d, 0xe6, 0x90, 0x5d, 0x96, 0xfc, - 0x5d, 0xc7, 0x9e, 0x1d, 0x7b, 0x62, 0x33, 0xf5, 0xc8, 0x17, 0x70, 0x53, 0x66, 0x51, 0x2f, 0x64, - 0x3d, 0x16, 0x91, 0xd0, 0x52, 0xf1, 0x3a, 0xf0, 0xce, 0x38, 0xf2, 0xe7, 0xd8, 0x4b, 0xe2, 0x95, - 0x85, 0x5b, 0xa4, 0xbf, 0x25, 0xb9, 0xcd, 0x37, 0x85, 0x96, 0x6d, 0xa5, 0x44, 0x51, 0xd1, 0x0f, - 0xe0, 0x46, 0x3f, 0x65, 0xb6, 0x7c, 0xd2, 0xb7, 0x7c, 0xc2, 0x43, 0x6a, 0x47, 0x03, 0xab, 0xc6, - 0x95, 0xe7, 0x00, 0x6f, 0x49, 0x76, 0xf3, 0x7a, 0x3f, 0x7b, 0xa4, 0x24, 0xea, 0xff, 0xd5, 0x60, - 0xa1, 0xdc, 0x3c, 0x75, 0x19, 0x2e, 0x5c, 0x0a, 0x49, 0x14, 0x7b, 0x3c, 0x52, 0x57, 0xf1, 0xe4, - 0xb4, 0x33, 0x0b, 0xb4, 0x24, 0x0c, 0xab, 0x81, 0xf3, 0x9c, 0x79, 0xb1, 0x4f, 0xb6, 0x49, 0x98, - 0x5c, 0x9d, 0xba, 0xb6, 0x54, 0x7b, 0x13, 0xc3, 0xf5, 0x02, 0x2e, 0xb4, 0x00, 0x57, 0x06, 0xc1, - 0x60, 0x0d, 0xe2, 0x1f, 0xd2, 0xcb, 0x7e, 0xea, 0xa0, 0x59, 0xa8, 0xf9, 0xa4, 0x2f, 0x3c, 0x52, - 0x35, 0x93, 0x4f, 0x74, 0x13, 0x2e, 0xf6, 0x85, 0x92, 0x46, 0x6d, 0x41, 0x6b, 0x4f, 0x99, 0x6a, - 0xa5, 0x3f, 0x80, 0xb6, 0x08, 0xba, 0x8f, 0x45, 0x89, 0xda, 0xa5, 0x24, 0x7c, 0x96, 0x14, 0xa8, - 0x75, 0x51, 0x32, 0xe2, 0x30, 0x7b, 0xaf, 0xfa, 0xaf, 0x35, 0x58, 0x3c, 0x03, 0xb3, 0xf2, 0x52, - 0x00, 0x8d, 0xb2, 0xba, 0xa7, 0xe2, 0xc0, 0x28, 0x70, 0xdb, 0x24, 0xd5, 0xca, 0x3d, 0x37, 0x48, - 0x11, 0x8f, 0xbe, 0x08, 0xf7, 0x05, 0xb8, 0xb5, 0x24, 0x68, 0x4c, 0xcc, 0x49, 0xb9, 0x21, 0xbf, - 0xd2, 0x94, 0xd5, 0x13, 0x79, 0x95, 0x1d, 0x87, 0xf0, 0x56, 0x49, 0x4f, 0x50, 0x66, 0x74, 0x0a, - 0xcc, 0x98, 0xa0, 0x58, 0x59, 0x21, 0x83, 0x7b, 0x84, 0x45, 0xdf, 0x83, 0xb7, 0x05, 0xb0, 0x1d, - 0x8e, 0x39, 0xd9, 0x8f, 0xbd, 0x4f, 0x92, 0x3e, 0x90, 0xe6, 0xd5, 0x0a, 0x4c, 0x8b, 0xbe, 0x90, - 0xde, 0x79, 0x7d, 0xb9, 0x59, 0x70, 0xb4, 0x10, 0x79, 0xea, 0xa4, 0xb1, 0xc4, 0xe4, 0x52, 0xff, - 0xa3, 0x06, 0xcd, 0x22, 0xd5, 0xca, 0xca, 0x3d, 0x78, 0x43, 0xea, 0xee, 0x79, 0xd8, 0x26, 0x3e, - 0x09, 0xb8, 0x3a, 0x62, 0xb1, 0xe0, 0x88, 0x67, 0x2c, 0x70, 0x77, 0x49, 0xe8, 0x0b, 0x15, 0xdb, - 0xa9, 0x80, 0x3a, 0xf1, 0x2a, 0xcb, 0x51, 0xd1, 0x3c, 0xd4, 0xf7, 0xa9, 0xe7, 0x59, 0xd8, 0x4f, - 0x6a, 0xba, 0x88, 0xc9, 0x29, 0x13, 0x12, 0xd2, 0xaa, 0xa0, 0xa0, 0x39, 0xb8, 0xcc, 0x43, 0xea, - 0xba, 0x24, 0x24, 0x8e, 0x88, 0xce, 0x69, 0x73, 0x48, 0xd0, 0xef, 0xc3, 0x5d, 0x01, 0xfb, 0x59, - 0xa6, 0xa3, 0x15, 0x5e, 0xea, 0x97, 0x1a, 0xdc, 0x3b, 0x8d, 0x53, 0x19, 0xfb, 0x05, 0x5c, 0x2f, - 0x68, 0x90, 0xca, 0xe0, 0xbb, 0x45, 0x06, 0x8f, 0xa9, 0x54, 0xc6, 0x22, 0x6f, 0x6c, 0x47, 0xff, - 0x4a, 0x83, 0xdb, 0x3b, 0xa2, 0xdd, 0x09, 0xff, 0x74, 0x19, 0x3b, 0xfc, 0x4c, 0x76, 0xc9, 0xf4, - 0x22, 0xc7, 0x13, 0xb8, 0x36, 0x92, 0xc0, 0x5b, 0x70, 0x75, 0xd8, 0x07, 0x2d, 0xea, 0x24, 0xd5, - 0xad, 0x36, 0x5e, 0x3a, 0x33, 0x7d, 0xb3, 0xb3, 0x33, 0xf8, 0x7e, 0xea, 0x98, 0x33, 0x51, 0x66, - 0x15, 0xe9, 0x18, 0x5a, 0x65, 0x88, 0x94, 0x4b, 0xbe, 0x03, 0x97, 0x54, 0x2b, 0x57, 0x35, 0x6d, - 0xbe, 0xc0, 0x0d, 0x52, 0x87, 0x14, 0x4d, 0xe3, 0x4b, 0x49, 0xe9, 0xbf, 0xab, 0xc1, 0x95, 0xec, - 0x3e, 0xba, 0x03, 0x57, 0x64, 0xde, 0x1c, 0x10, 0xea, 0x1e, 0x70, 0x55, 0xa5, 0xea, 0x82, 0xb6, - 0x29, 0x48, 0xe8, 0x16, 0x5c, 0x26, 0xc7, 0xc4, 0xb6, 0x7c, 0xe6, 0x10, 0x11, 0x18, 0x33, 0xe6, - 0x74, 0x42, 0xd8, 0x62, 0x0e, 0x41, 0x9f, 0xc1, 0x2c, 0x4b, 0xd1, 0xaa, 0x31, 0x43, 0x44, 0x47, - 0x7d, 0xb9, 0x5d, 0x0a, 0x6d, 0xc4, 0xbc, 0xcd, 0x8a, 0xf9, 0x06, 0xcb, 0x93, 0x92, 0x4e, 0x28, - 0x03, 0x3d, 0x89, 0xc0, 0xc6, 0x54, 0x69, 0x43, 0x1a, 0x51, 0xb8, 0x41, 0x3d, 0x6f, 0xb3, 0x62, - 0x5e, 0x16, 0xb2, 0xc9, 0x02, 0x6d, 0x40, 0x9d, 0xe3, 0x43, 0x12, 0x5a, 0x82, 0xd4, 0xb8, 0x20, - 0x34, 0xbd, 0x53, 0xaa, 0x69, 0x37, 0xe1, 0x15, 0xea, 0x36, 0x2b, 0x26, 0xf0, 0xc1, 0x0a, 0x59, - 0x70, 0x2d, 0x73, 0xd5, 0xca, 0xd0, 0x8b, 0x42, 0xdb, 0xd2, 0x84, 0xdb, 0x16, 0x4a, 0x87, 0x77, - 0x3e, 0x30, 0x78, 0x36, 0x1a, 0xa1, 0xad, 0xcd, 0xc2, 0x55, 0xa9, 0xd5, 0xf2, 0x49, 0x14, 0x61, - 0x97, 0xe8, 0x7f, 0xd3, 0xa0, 0xb1, 0xc3, 0xb1, 0x4b, 0x9c, 0x0d, 0x1a, 0x60, 0x8f, 0xfe, 0x90, - 0x88, 0x7a, 0xf5, 0x71, 0x3f, 0xc9, 0xd7, 0xbc, 0x83, 0xb4, 0xf3, 0x3b, 0xa8, 0xd0, 0xb0, 0xea, - 0xff, 0xd1, 0xb0, 0x4b, 0x70, 0x81, 0x24, 0x90, 0xf5, 0x5f, 0x68, 0x70, 0xa3, 0x30, 0x00, 0x50, - 0x13, 0xa6, 0xa3, 0x00, 0xf7, 0xa2, 0x03, 0x26, 0x03, 0x70, 0xda, 0x1c, 0xac, 0xd1, 0xde, 0x30, - 0xe4, 0x65, 0x72, 0x7d, 0x98, 0x47, 0xa5, 0xc6, 0xdf, 0xce, 0xf8, 0xb0, 0xfb, 0xc9, 0xfe, 0xfe, - 0x7a, 0x42, 0x90, 0x87, 0x3c, 0x7f, 0x34, 0x9a, 0x0b, 0xbf, 0xd7, 0xe0, 0x7a, 0x81, 0x7b, 0xd0, - 0x0a, 0x88, 0x1c, 0x97, 0xe3, 0x90, 0x72, 0xed, 0x5c, 0xc9, 0x18, 0x27, 0xc6, 0x1d, 0x53, 0x4c, - 0x7d, 0xe2, 0x13, 0x7d, 0x00, 0x17, 0x85, 0x6f, 0x53, 0xb4, 0x8d, 0xb2, 0xda, 0xaf, 0xd0, 0x28, - 0xee, 0x24, 0x0f, 0x33, 0xf5, 0x37, 0x6a, 0xd4, 0x16, 0x6a, 0xed, 0x29, 0xb3, 0x3e, 0x2c, 0xc0, - 0x91, 0xfe, 0x65, 0x15, 0x66, 0x47, 0xa3, 0x14, 0x2d, 0xc1, 0x05, 0x19, 0xd9, 0x12, 0x67, 0xe9, - 0x71, 0x9b, 0x15, 0x53, 0x32, 0xa2, 0x3d, 0xb8, 0x96, 0x29, 0x87, 0x2a, 0x2f, 0xaa, 0xa5, 0x5d, - 0x44, 0x9e, 0x98, 0x29, 0xad, 0xa9, 0xba, 0x59, 0x6f, 0x84, 0x86, 0x3e, 0x07, 0x94, 0xc9, 0x35, - 0x2b, 0xe2, 0x98, 0xc7, 0x91, 0xaa, 0x06, 0x8b, 0x67, 0x48, 0xb9, 0x1d, 0x21, 0x60, 0xce, 0xf2, - 0x11, 0xca, 0xda, 0x4c, 0x2e, 0x89, 0xf5, 0x3f, 0x68, 0x70, 0xb3, 0x58, 0x36, 0x71, 0x63, 0xee, - 0x70, 0x55, 0xce, 0x58, 0x86, 0xe5, 0x21, 0xa0, 0x90, 0xf8, 0x98, 0x06, 0x34, 0x70, 0xad, 0xa3, - 0x18, 0x07, 0x3c, 0xf6, 0x23, 0xd5, 0xf0, 0xae, 0x0d, 0x76, 0x3e, 0x55, 0x1b, 0xe8, 0xbb, 0xd0, - 0x62, 0x3d, 0x4e, 0x7d, 0x1a, 0x71, 0x6a, 0x63, 0xcf, 0x3b, 0x11, 0x19, 0x47, 0x9c, 0xa1, 0xa8, - 0x1c, 0xd5, 0xe6, 0xf2, 0x5c, 0x1b, 0x82, 0x29, 0xd5, 0xb2, 0xfc, 0x1b, 0x80, 0x0b, 0xa2, 0xed, - 0xa1, 0x9f, 0x69, 0x30, 0x9d, 0x3e, 0x00, 0xd0, 0x83, 0x02, 0xaf, 0x94, 0xbc, 0xa2, 0x9a, 0xed, - 0x32, 0xde, 0xd1, 0x67, 0x94, 0xbe, 0xf8, 0xd3, 0xbf, 0xff, 0xe7, 0x97, 0xd5, 0x77, 0xd0, 0x1d, - 0x63, 0xc2, 0x3b, 0xd8, 0xf8, 0x11, 0x75, 0x7e, 0x8c, 0x7e, 0xae, 0x41, 0x3d, 0xf3, 0x92, 0x29, - 0x07, 0x34, 0xfe, 0xa4, 0x6a, 0xbe, 0x7b, 0x1a, 0xa0, 0xcc, 0xd3, 0x48, 0xff, 0x86, 0xc0, 0xd4, - 0x42, 0x73, 0x93, 0x30, 0xa1, 0x3f, 0x6b, 0xd0, 0x28, 0x1b, 0xc9, 0xd1, 0xf2, 0x6b, 0xcd, 0xef, - 0x12, 0xe3, 0x7b, 0xe7, 0x98, 0xf9, 0xf5, 0xc7, 0x02, 0xeb, 0xfb, 0x8f, 0xb5, 0x07, 0xba, 0x61, - 0x14, 0x3e, 0xc4, 0xad, 0x80, 0x39, 0xc4, 0xe2, 0x4c, 0xfe, 0xb7, 0x33, 0x20, 0xff, 0xaa, 0xc1, - 0xdc, 0xa4, 0xe9, 0x18, 0xad, 0x94, 0x79, 0xed, 0x0c, 0xb3, 0x7d, 0xf3, 0xdb, 0xe7, 0x13, 0x56, - 0x76, 0xdd, 0x13, 0x76, 0x2d, 0xa0, 0x96, 0x31, 0xf1, 0xc7, 0x0f, 0xf4, 0x27, 0x0d, 0x6e, 0x4d, - 0x18, 0x8d, 0xd1, 0xe3, 0x32, 0x14, 0xa7, 0x0f, 0xf5, 0xcd, 0x95, 0x73, 0xc9, 0x2a, 0x03, 0xee, - 0x0a, 0x03, 0xe6, 0xd1, 0xed, 0x89, 0xbf, 0x08, 0xa1, 0xbf, 0x68, 0xf0, 0x76, 0xe9, 0x78, 0x89, - 0x3e, 0x2c, 0x43, 0x70, 0xda, 0xec, 0xda, 0xfc, 0xd6, 0x39, 0x24, 0x15, 0xf2, 0x8e, 0x40, 0xde, - 0x46, 0xf7, 0x8c, 0x33, 0xfd, 0x0a, 0x84, 0x02, 0x98, 0xc9, 0xbd, 0x00, 0xd0, 0x37, 0xcb, 0xce, - 0x2e, 0x7a, 0x83, 0x34, 0x1f, 0x9e, 0x91, 0x5b, 0xa1, 0xab, 0xa0, 0x9f, 0xa4, 0x15, 0x75, 0x74, - 0xf4, 0x44, 0x4b, 0x67, 0x1d, 0xe3, 0xd2, 0xb9, 0xb9, 0xf9, 0xe8, 0x35, 0x24, 0x24, 0x80, 0x25, - 0x6d, 0x6d, 0xfb, 0xeb, 0x97, 0x2d, 0xed, 0xc5, 0xcb, 0x96, 0xf6, 0xef, 0x97, 0x2d, 0xed, 0xab, - 0x57, 0xad, 0xca, 0x8b, 0x57, 0xad, 0xca, 0x3f, 0x5e, 0xb5, 0x2a, 0xdf, 0xff, 0xc0, 0xa5, 0xfc, - 0x20, 0xee, 0x76, 0x6c, 0xe6, 0xe7, 0x9d, 0xd7, 0x7f, 0xff, 0xa1, 0x68, 0xf8, 0xc6, 0x80, 0x72, - 0x2c, 0x1d, 0xca, 0x4f, 0x7a, 0x24, 0xea, 0x5e, 0x14, 0xe4, 0xf7, 0xfe, 0x17, 0x00, 0x00, 0xff, - 0xff, 0x8b, 0xcc, 0x03, 0x3a, 0xd0, 0x14, 0x00, 0x00, + // 1661 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x58, 0xc1, 0x4f, 0xdc, 0x46, + 0x17, 0x5f, 0xb3, 0x84, 0xc0, 0xdb, 0x40, 0x60, 0x08, 0xc9, 0x66, 0x21, 0x0b, 0x71, 0xbe, 0x90, + 0x85, 0x7c, 0x59, 0x03, 0x89, 0xa2, 0x7c, 0xe1, 0x53, 0x3e, 0x01, 0xdf, 0x47, 0x88, 0x14, 0xbe, + 0x10, 0x43, 0x12, 0xd4, 0x46, 0xb2, 0xbc, 0xf6, 0xb0, 0x58, 0xd8, 0x9e, 0xc5, 0x1e, 0xaf, 0x40, + 0x55, 0x55, 0xa9, 0x87, 0x5c, 0xda, 0x4a, 0x91, 0x7a, 0xe8, 0xa1, 0x52, 0x2f, 0x3d, 0xf5, 0x50, + 0xa9, 0x97, 0x1e, 0xab, 0xb6, 0xb7, 0x1c, 0x23, 0xf5, 0xd2, 0x43, 0x55, 0x55, 0x49, 0xcf, 0xfd, + 0x1b, 0x2a, 0xcf, 0x8c, 0x77, 0xbd, 0xbb, 0xf6, 0x42, 0xb8, 0x80, 0xfd, 0xe6, 0xbd, 0x37, 0xbf, + 0xf7, 0xde, 0x6f, 0xde, 0x3c, 0x2f, 0x5c, 0x32, 0x0f, 0xcd, 0x83, 0x9a, 0x47, 0x28, 0x31, 0x88, + 0xad, 0x18, 0x36, 0xa9, 0x28, 0xfb, 0x01, 0xf6, 0x0e, 0xcb, 0x4c, 0x86, 0x46, 0xe2, 0xcb, 0xe5, + 0x70, 0xb9, 0x70, 0xae, 0x4a, 0xaa, 0x84, 0x89, 0x94, 0xf0, 0x89, 0x2b, 0x16, 0x26, 0xaa, 0x84, + 0x54, 0x6d, 0xac, 0xe8, 0x35, 0x4b, 0xd1, 0x5d, 0x97, 0x50, 0x9d, 0x5a, 0xc4, 0xf5, 0xc5, 0xea, + 0xac, 0x41, 0x7c, 0x87, 0xf8, 0x4a, 0x45, 0xf7, 0x31, 0xf7, 0xaf, 0xd4, 0xe7, 0x2b, 0x98, 0xea, + 0xf3, 0x4a, 0x4d, 0xaf, 0x5a, 0x2e, 0x53, 0x16, 0xba, 0x4a, 0x27, 0xa2, 0x8a, 0x4d, 0x8c, 0x3d, + 0xcd, 0xd3, 0x29, 0xd6, 0x6c, 0xcb, 0xb1, 0xa8, 0x66, 0x10, 0x77, 0xc7, 0xaa, 0x0a, 0x83, 0xcb, + 0x9d, 0x06, 0xe1, 0x1f, 0xad, 0xa6, 0x5b, 0x9e, 0x50, 0x99, 0xeb, 0x54, 0xc1, 0xfb, 0x81, 0x45, + 0x0f, 0x35, 0x6a, 0x61, 0x2f, 0xc9, 0x69, 0x42, 0x5e, 0x88, 0x67, 0xe2, 0xc8, 0xe1, 0x64, 0xe7, + 0xb2, 0xa3, 0x53, 0x63, 0x17, 0x47, 0x11, 0x5f, 0xef, 0x54, 0xb0, 0xad, 0xfd, 0xc0, 0x32, 0x79, + 0x5e, 0x5a, 0x37, 0x1b, 0x4f, 0xf0, 0x86, 0xeb, 0x62, 0xf1, 0x5e, 0xcb, 0xa2, 0xe5, 0x9a, 0xf8, + 0x00, 0x7b, 0x0a, 0xd9, 0xd9, 0xd1, 0x8c, 0x5d, 0xdd, 0x72, 0xb5, 0xa0, 0x66, 0xea, 0x14, 0xfb, + 0x9d, 0x12, 0x61, 0x5f, 0x6a, 0xb1, 0xf7, 0x83, 0x8a, 0x6e, 0x18, 0x24, 0x70, 0xa9, 0xaf, 0xf8, + 0xd4, 0xc3, 0xba, 0x63, 0xb9, 0x11, 0x8c, 0x99, 0x74, 0xcd, 0xc6, 0x33, 0x57, 0x95, 0x67, 0xe0, + 0xc2, 0xe3, 0xb0, 0x8c, 0xf7, 0x31, 0x5d, 0xb1, 0x49, 0x65, 0x43, 0xb7, 0x3c, 0x15, 0xef, 0x07, + 0xd8, 0xa7, 0x68, 0x08, 0x7a, 0x2c, 0x33, 0x2f, 0x4d, 0x49, 0xa5, 0x41, 0xb5, 0xc7, 0x32, 0xe5, + 0x67, 0x30, 0xc6, 0x54, 0x9b, 0x7a, 0x7e, 0x8d, 0xb8, 0x3e, 0x46, 0xf7, 0x60, 0xa0, 0x51, 0x27, + 0xa6, 0x9f, 0x5b, 0x18, 0x2f, 0x77, 0xf0, 0xad, 0x1c, 0xd9, 0x2d, 0xf7, 0xbe, 0xfa, 0x7d, 0x32, + 0xa3, 0xf6, 0x1b, 0xe2, 0x5d, 0xd6, 0x05, 0x86, 0x25, 0xdb, 0x6e, 0xc7, 0xb0, 0x0a, 0xd0, 0xe4, + 0x95, 0xf0, 0x3d, 0x5d, 0xe6, 0x24, 0x2c, 0x87, 0x24, 0x2c, 0x73, 0x92, 0x0b, 0x12, 0x96, 0x37, + 0xf4, 0x2a, 0x16, 0xb6, 0x6a, 0xcc, 0x52, 0xfe, 0x5a, 0x82, 0x7c, 0x0b, 0xf8, 0x25, 0xdb, 0x4e, + 0xc3, 0x9f, 0x7d, 0x47, 0xfc, 0xe8, 0x7e, 0x0b, 0xc8, 0x1e, 0x06, 0xf2, 0xda, 0x91, 0x20, 0xf9, + 0xe6, 0x2d, 0x28, 0x7f, 0x93, 0x60, 0x72, 0x1d, 0xd7, 0xff, 0x4f, 0x4c, 0xbc, 0x45, 0xc2, 0xbf, + 0x2b, 0xba, 0x6d, 0x04, 0x36, 0x5b, 0x8c, 0x32, 0xf2, 0x1c, 0xce, 0xf3, 0x53, 0x54, 0xf3, 0x48, + 0x8d, 0xf8, 0xd8, 0xd3, 0x04, 0x5f, 0x1b, 0xd9, 0xe9, 0x44, 0xfe, 0x54, 0xb7, 0x43, 0xbe, 0x12, + 0x6f, 0x1d, 0xd7, 0xd7, 0xb9, 0xb6, 0x7a, 0x8e, 0x79, 0xd9, 0x10, 0x4e, 0x84, 0x14, 0xbd, 0x0f, + 0x63, 0xf5, 0x48, 0x59, 0x73, 0x70, 0x5d, 0x73, 0x30, 0xf5, 0x2c, 0xc3, 0x6f, 0x44, 0xd5, 0xe9, + 0xbc, 0x05, 0xf0, 0x3a, 0x57, 0x57, 0x47, 0xeb, 0xf1, 0x2d, 0xb9, 0x50, 0xfe, 0x4b, 0x82, 0xa9, + 0xf4, 0xf0, 0x44, 0x31, 0xaa, 0x70, 0xda, 0xc3, 0x7e, 0x60, 0x53, 0x5f, 0x94, 0xe2, 0xfe, 0x51, + 0x7b, 0x26, 0x78, 0x09, 0x15, 0x96, 0x5c, 0xf3, 0x29, 0xb1, 0x03, 0x07, 0x6f, 0x60, 0x2f, 0x2c, + 0x9d, 0x28, 0x5b, 0xe4, 0xbd, 0xa0, 0xc3, 0x68, 0x82, 0x16, 0x9a, 0x82, 0x33, 0x0d, 0x32, 0x68, + 0x0d, 0xfe, 0x43, 0x54, 0xec, 0x07, 0x26, 0x1a, 0x86, 0xac, 0x83, 0xeb, 0x2c, 0x23, 0x3d, 0x6a, + 0xf8, 0x88, 0xce, 0x43, 0x5f, 0x9d, 0x39, 0xc9, 0x67, 0xa7, 0xa4, 0x52, 0xaf, 0x2a, 0xde, 0xe4, + 0x59, 0x28, 0x31, 0xd2, 0xfd, 0x8f, 0xb5, 0xa8, 0x2d, 0x0b, 0x7b, 0x0f, 0xc3, 0x06, 0xb5, 0xc2, + 0x5a, 0x46, 0xe0, 0xc5, 0xeb, 0x2a, 0x7f, 0x29, 0xc1, 0xcc, 0x31, 0x94, 0x45, 0x96, 0x5c, 0xc8, + 0xa7, 0xf5, 0x3d, 0xc1, 0x03, 0x25, 0x21, 0x6d, 0xdd, 0x5c, 0x8b, 0xf4, 0x8c, 0xe1, 0x24, 0x1d, + 0x79, 0x06, 0xae, 0x31, 0x70, 0xcb, 0x21, 0x69, 0x54, 0x9d, 0xe2, 0xf4, 0x40, 0xbe, 0x90, 0x44, + 0xd4, 0x5d, 0x75, 0x45, 0x1c, 0x7b, 0x70, 0x21, 0xe5, 0x4e, 0x10, 0x61, 0x94, 0x13, 0xc2, 0xe8, + 0xe2, 0x58, 0x44, 0xc1, 0xc9, 0xdd, 0xa6, 0x22, 0x6f, 0xc3, 0x45, 0x06, 0x6c, 0x93, 0xea, 0x14, + 0xef, 0x04, 0xf6, 0xa3, 0xf0, 0x1e, 0x88, 0xce, 0xd5, 0x22, 0xf4, 0xb3, 0x7b, 0x21, 0xaa, 0x79, + 0x6e, 0xa1, 0x90, 0xb0, 0x35, 0x33, 0x79, 0x60, 0x46, 0x5c, 0x22, 0xfc, 0x55, 0xfe, 0x5e, 0x82, + 0x42, 0x92, 0x6b, 0x11, 0xe5, 0x36, 0x9c, 0xe5, 0xbe, 0x6b, 0xb6, 0x6e, 0x60, 0x07, 0xbb, 0x54, + 0x6c, 0x31, 0x93, 0xb0, 0xc5, 0x43, 0xe2, 0x56, 0xb7, 0xb0, 0xe7, 0x30, 0x17, 0x1b, 0x91, 0x81, + 0xd8, 0x71, 0x88, 0xb4, 0x48, 0xd1, 0x24, 0xe4, 0x76, 0x2c, 0xdb, 0xd6, 0x74, 0x27, 0xec, 0xe9, + 0x8c, 0x93, 0xbd, 0x2a, 0x84, 0xa2, 0x25, 0x26, 0x41, 0x13, 0x30, 0x40, 0x3d, 0xab, 0x5a, 0xc5, + 0x1e, 0x36, 0x19, 0x3b, 0xfb, 0xd5, 0xa6, 0x40, 0xbe, 0x06, 0x57, 0x19, 0xec, 0x87, 0xb1, 0x1b, + 0x2d, 0xb1, 0xa8, 0x2f, 0x24, 0x98, 0x3e, 0x4a, 0x53, 0x04, 0xfb, 0x1c, 0x46, 0x13, 0x2e, 0x48, + 0x11, 0xf0, 0xd5, 0xa4, 0x80, 0x3b, 0x5c, 0x8a, 0x60, 0x91, 0xdd, 0xb1, 0x22, 0xbf, 0x94, 0xe0, + 0xd2, 0x26, 0xbb, 0xee, 0x58, 0x7e, 0x2a, 0x84, 0xec, 0x3d, 0xe1, 0xb7, 0x64, 0x54, 0xc8, 0xce, + 0x03, 0x9c, 0x6d, 0x3b, 0xc0, 0xeb, 0x30, 0xd4, 0xbc, 0x07, 0x35, 0xcb, 0x0c, 0xbb, 0x5b, 0xb6, + 0xb3, 0x75, 0xc6, 0xee, 0xcd, 0xf2, 0x66, 0xe3, 0xf9, 0x81, 0xa9, 0x0e, 0xfa, 0xb1, 0x37, 0x5f, + 0xd6, 0xa1, 0x98, 0x86, 0x48, 0xa4, 0xe4, 0x3f, 0x70, 0x5a, 0x5c, 0xe5, 0xa2, 0xa7, 0x4d, 0x26, + 0xa4, 0x81, 0xfb, 0xe0, 0xa6, 0x11, 0xbf, 0x84, 0x95, 0xfc, 0x4d, 0x16, 0xce, 0xc4, 0xd7, 0xd1, + 0x65, 0x38, 0xc3, 0xcf, 0xcd, 0x2e, 0xb6, 0xaa, 0xbb, 0x54, 0x74, 0xa9, 0x1c, 0x93, 0xad, 0x31, + 0x11, 0x1a, 0x87, 0x01, 0x7c, 0x80, 0x0d, 0xcd, 0x21, 0x26, 0x66, 0xc4, 0x18, 0x54, 0xfb, 0x43, + 0xc1, 0x3a, 0x31, 0x31, 0x7a, 0x02, 0xc3, 0x24, 0x42, 0x2b, 0xc6, 0x0c, 0xc6, 0x8e, 0xdc, 0x42, + 0x29, 0x15, 0x5a, 0x5b, 0x78, 0x6b, 0x19, 0xf5, 0x2c, 0x69, 0x15, 0x85, 0x37, 0x21, 0x27, 0x7a, + 0xc8, 0xc0, 0x7c, 0x6f, 0xea, 0x85, 0xd4, 0xe6, 0x70, 0xd5, 0xb2, 0xed, 0xb5, 0x8c, 0x3a, 0xc0, + 0x6c, 0xc3, 0x17, 0xb4, 0x0a, 0x39, 0xaa, 0xef, 0x61, 0x4f, 0x63, 0xa2, 0xfc, 0x29, 0xe6, 0xe9, + 0x4a, 0xaa, 0xa7, 0xad, 0x50, 0x97, 0xb9, 0x5b, 0xcb, 0xa8, 0x40, 0x1b, 0x6f, 0x48, 0x83, 0x91, + 0x58, 0xa9, 0x45, 0xa0, 0x7d, 0xcc, 0xdb, 0x5c, 0x97, 0x6a, 0x33, 0xa7, 0xcd, 0x9a, 0x37, 0x02, + 0x1e, 0xf6, 0xdb, 0x64, 0xcb, 0xc3, 0x30, 0xc4, 0xbd, 0x6a, 0x0e, 0xf6, 0x7d, 0xbd, 0x8a, 0xe5, + 0xcf, 0x24, 0x18, 0x4b, 0x4c, 0x18, 0x2a, 0x40, 0xbf, 0xef, 0xea, 0x35, 0x7f, 0x97, 0xf0, 0x82, + 0xf5, 0xab, 0x8d, 0x77, 0xb4, 0xdd, 0xa4, 0x08, 0x27, 0xe3, 0x9d, 0x56, 0x78, 0x62, 0x5c, 0x2c, + 0x77, 0x0e, 0x87, 0x8f, 0x76, 0x76, 0x56, 0x42, 0x01, 0xdf, 0xe4, 0xe9, 0x7c, 0x3b, 0x77, 0xbe, + 0x95, 0x60, 0x34, 0x21, 0xdf, 0x68, 0x11, 0xd8, 0x99, 0xe0, 0xe3, 0x83, 0x38, 0x9e, 0x13, 0x29, + 0x63, 0x0f, 0x1b, 0x0f, 0x54, 0x36, 0x25, 0xb1, 0x47, 0x74, 0x1b, 0xfa, 0x58, 0x65, 0x22, 0xb4, + 0xf9, 0xb4, 0x5e, 0x29, 0xd0, 0x08, 0xed, 0x90, 0xb7, 0xb1, 0x7e, 0xe5, 0xe7, 0xb3, 0x53, 0xd9, + 0x52, 0xaf, 0x9a, 0x6b, 0x36, 0x2c, 0x5f, 0x7e, 0xd1, 0x03, 0xc3, 0xed, 0x55, 0x45, 0x73, 0x70, + 0x8a, 0x33, 0x81, 0xe3, 0x4c, 0xdd, 0x6e, 0x2d, 0xa3, 0x72, 0x45, 0xb4, 0x0d, 0x23, 0xb1, 0xf6, + 0x21, 0x78, 0xd4, 0x93, 0xda, 0x75, 0xf9, 0x8e, 0xb1, 0x56, 0x14, 0xb9, 0x1b, 0xb6, 0xdb, 0x64, + 0xe8, 0x19, 0xa0, 0x18, 0x37, 0x35, 0x9f, 0xea, 0x34, 0xf0, 0xc5, 0xe9, 0x99, 0x39, 0x06, 0x45, + 0x37, 0x99, 0x81, 0x3a, 0x4c, 0xdb, 0x24, 0xcb, 0x83, 0x2d, 0xa4, 0x97, 0xbf, 0x93, 0xe0, 0x7c, + 0xb2, 0x6d, 0x98, 0xc6, 0x96, 0xcd, 0xc5, 0xf1, 0x27, 0x31, 0x95, 0x1b, 0x80, 0x3c, 0xec, 0xe8, + 0x96, 0x6b, 0xb9, 0x55, 0x6d, 0x3f, 0xd0, 0x5d, 0x1a, 0x38, 0xbe, 0xb8, 0x20, 0x46, 0x1a, 0x2b, + 0x8f, 0xc5, 0x02, 0xfa, 0x2f, 0x14, 0x49, 0x8d, 0x5a, 0x8e, 0xe5, 0x53, 0xcb, 0xd0, 0x6d, 0xfb, + 0x90, 0x1d, 0x61, 0x6c, 0x36, 0x4d, 0xf9, 0x68, 0x33, 0xd1, 0xaa, 0xb5, 0xca, 0x94, 0x22, 0x2f, + 0x0b, 0x5f, 0x01, 0x9c, 0x62, 0xd7, 0x04, 0xfa, 0x44, 0x82, 0xfe, 0x68, 0x60, 0x46, 0xb3, 0x09, + 0x59, 0x49, 0xf9, 0xea, 0x28, 0x94, 0xd2, 0x74, 0xdb, 0x3f, 0x3b, 0xe4, 0x99, 0x8f, 0x7f, 0xf9, + 0xf3, 0xf3, 0x9e, 0x2b, 0xe8, 0xb2, 0xd2, 0xe5, 0xbb, 0x51, 0xf9, 0xc0, 0x32, 0x3f, 0x44, 0x9f, + 0x4a, 0x90, 0x8b, 0x4d, 0xfe, 0xe9, 0x80, 0x3a, 0x3f, 0x41, 0x0a, 0xd7, 0x8f, 0x02, 0x14, 0xfb, + 0x94, 0x90, 0xff, 0xc1, 0x30, 0x15, 0xd1, 0x44, 0x37, 0x4c, 0xe8, 0x47, 0x09, 0xf2, 0x69, 0x23, + 0x2c, 0x5a, 0x78, 0xa7, 0x79, 0x97, 0x63, 0xbc, 0x79, 0x82, 0x19, 0x59, 0xbe, 0xcb, 0xb0, 0xde, + 0xba, 0x2b, 0xcd, 0xca, 0x8a, 0x92, 0xf8, 0xe1, 0xaa, 0xb9, 0xc4, 0xc4, 0x1a, 0x25, 0xfc, 0xbf, + 0x11, 0x03, 0xf9, 0xb3, 0x04, 0x13, 0xdd, 0xa6, 0x49, 0xb4, 0x98, 0x96, 0xb5, 0x63, 0xcc, 0xc2, + 0x85, 0x7f, 0x9f, 0xcc, 0x58, 0xc4, 0x35, 0xcd, 0xe2, 0x9a, 0x42, 0x45, 0xa5, 0xeb, 0x8f, 0x05, + 0xe8, 0x07, 0x09, 0xc6, 0xbb, 0x8c, 0x92, 0xe8, 0x6e, 0x1a, 0x8a, 0xa3, 0x87, 0xe0, 0xc2, 0xe2, + 0x89, 0x6c, 0x45, 0x00, 0x57, 0x59, 0x00, 0x93, 0xe8, 0x52, 0xd7, 0x5f, 0x50, 0xd0, 0x4f, 0x12, + 0x5c, 0x4c, 0x1d, 0xc7, 0xd0, 0x9d, 0x34, 0x04, 0x47, 0xcd, 0x7a, 0x85, 0x7f, 0x9d, 0xc0, 0x52, + 0x20, 0x2f, 0x33, 0xe4, 0x25, 0x34, 0xad, 0x1c, 0xeb, 0x57, 0x13, 0xe4, 0xc2, 0x60, 0xcb, 0xc4, + 0x8c, 0xfe, 0x99, 0xb6, 0x77, 0xd2, 0xcc, 0x5e, 0xb8, 0x71, 0x4c, 0x6d, 0x81, 0x2e, 0x83, 0x3e, + 0x8a, 0x3a, 0x6a, 0xfb, 0xa8, 0x86, 0xe6, 0x8e, 0x3b, 0xf6, 0x44, 0x73, 0x66, 0x61, 0xfe, 0x1d, + 0x2c, 0x38, 0x80, 0x39, 0x69, 0x79, 0xe3, 0xd5, 0x9b, 0xa2, 0xf4, 0xfa, 0x4d, 0x51, 0xfa, 0xe3, + 0x4d, 0x51, 0x7a, 0xf9, 0xb6, 0x98, 0x79, 0xfd, 0xb6, 0x98, 0xf9, 0xf5, 0x6d, 0x31, 0xf3, 0xde, + 0xed, 0xaa, 0x45, 0x77, 0x83, 0x4a, 0xd9, 0x20, 0x4e, 0x6b, 0xf2, 0xea, 0xb7, 0x6e, 0xb0, 0x0b, + 0x5f, 0x69, 0x48, 0x0e, 0x78, 0x42, 0xe9, 0x61, 0x0d, 0xfb, 0x95, 0x3e, 0x26, 0xbe, 0xf9, 0x77, + 0x00, 0x00, 0x00, 0xff, 0xff, 0xa1, 0x46, 0x6b, 0x53, 0x00, 0x14, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -2646,80 +2553,6 @@ func (m *StreamUpdate_SubaccountUpdate) MarshalToSizedBuffer(dAtA []byte) (int, } return len(dAtA) - i, nil } -func (m *StagedFinalizeBlockEvent) Marshal() (dAtA []byte, err error) { - size := m.Size() - dAtA = make([]byte, size) - n, err := m.MarshalToSizedBuffer(dAtA[:size]) - if err != nil { - return nil, err - } - return dAtA[:n], nil -} - -func (m *StagedFinalizeBlockEvent) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *StagedFinalizeBlockEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - _ = i - var l int - _ = l - if m.Event != nil { - { - size := m.Event.Size() - i -= size - if _, err := m.Event.MarshalTo(dAtA[i:]); err != nil { - return 0, err - } - } - } - return len(dAtA) - i, nil -} - -func (m *StagedFinalizeBlockEvent_OrderFill) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *StagedFinalizeBlockEvent_OrderFill) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.OrderFill != nil { - { - size, err := m.OrderFill.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQuery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0xa - } - return len(dAtA) - i, nil -} -func (m *StagedFinalizeBlockEvent_SubaccountUpdate) MarshalTo(dAtA []byte) (int, error) { - size := m.Size() - return m.MarshalToSizedBuffer(dAtA[:size]) -} - -func (m *StagedFinalizeBlockEvent_SubaccountUpdate) MarshalToSizedBuffer(dAtA []byte) (int, error) { - i := len(dAtA) - if m.SubaccountUpdate != nil { - { - size, err := m.SubaccountUpdate.MarshalToSizedBuffer(dAtA[:i]) - if err != nil { - return 0, err - } - i -= size - i = encodeVarintQuery(dAtA, i, uint64(size)) - } - i-- - dAtA[i] = 0x12 - } - return len(dAtA) - i, nil -} func (m *StreamOrderbookUpdate) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -2788,20 +2621,20 @@ func (m *StreamOrderbookFill) MarshalToSizedBuffer(dAtA []byte) (int, error) { var l int _ = l if len(m.FillAmounts) > 0 { - dAtA20 := make([]byte, len(m.FillAmounts)*10) - var j19 int + dAtA18 := make([]byte, len(m.FillAmounts)*10) + var j17 int for _, num := range m.FillAmounts { for num >= 1<<7 { - dAtA20[j19] = uint8(uint64(num)&0x7f | 0x80) + dAtA18[j17] = uint8(uint64(num)&0x7f | 0x80) num >>= 7 - j19++ + j17++ } - dAtA20[j19] = uint8(num) - j19++ + dAtA18[j17] = uint8(num) + j17++ } - i -= j19 - copy(dAtA[i:], dAtA20[:j19]) - i = encodeVarintQuery(dAtA, i, uint64(j19)) + i -= j17 + copy(dAtA[i:], dAtA18[:j17]) + i = encodeVarintQuery(dAtA, i, uint64(j17)) i-- dAtA[i] = 0x1a } @@ -3265,42 +3098,6 @@ func (m *StreamUpdate_SubaccountUpdate) Size() (n int) { } return n } -func (m *StagedFinalizeBlockEvent) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.Event != nil { - n += m.Event.Size() - } - return n -} - -func (m *StagedFinalizeBlockEvent_OrderFill) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.OrderFill != nil { - l = m.OrderFill.Size() - n += 1 + l + sovQuery(uint64(l)) - } - return n -} -func (m *StagedFinalizeBlockEvent_SubaccountUpdate) Size() (n int) { - if m == nil { - return 0 - } - var l int - _ = l - if m.SubaccountUpdate != nil { - l = m.SubaccountUpdate.Size() - n += 1 + l + sovQuery(uint64(l)) - } - return n -} func (m *StreamOrderbookUpdate) Size() (n int) { if m == nil { return 0 @@ -5148,126 +4945,6 @@ func (m *StreamUpdate) Unmarshal(dAtA []byte) error { } return nil } -func (m *StagedFinalizeBlockEvent) Unmarshal(dAtA []byte) error { - l := len(dAtA) - iNdEx := 0 - for iNdEx < l { - preIndex := iNdEx - var wire uint64 - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQuery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - wire |= uint64(b&0x7F) << shift - if b < 0x80 { - break - } - } - fieldNum := int32(wire >> 3) - wireType := int(wire & 0x7) - if wireType == 4 { - return fmt.Errorf("proto: StagedFinalizeBlockEvent: wiretype end group for non-group") - } - if fieldNum <= 0 { - return fmt.Errorf("proto: StagedFinalizeBlockEvent: illegal tag %d (wire type %d)", fieldNum, wire) - } - switch fieldNum { - case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field OrderFill", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQuery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQuery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQuery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - v := &StreamOrderbookFill{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - m.Event = &StagedFinalizeBlockEvent_OrderFill{v} - iNdEx = postIndex - case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field SubaccountUpdate", wireType) - } - var msglen int - for shift := uint(0); ; shift += 7 { - if shift >= 64 { - return ErrIntOverflowQuery - } - if iNdEx >= l { - return io.ErrUnexpectedEOF - } - b := dAtA[iNdEx] - iNdEx++ - msglen |= int(b&0x7F) << shift - if b < 0x80 { - break - } - } - if msglen < 0 { - return ErrInvalidLengthQuery - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthQuery - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - v := &types.StreamSubaccountUpdate{} - if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - m.Event = &StagedFinalizeBlockEvent_SubaccountUpdate{v} - iNdEx = postIndex - default: - iNdEx = preIndex - skippy, err := skipQuery(dAtA[iNdEx:]) - if err != nil { - return err - } - if (skippy < 0) || (iNdEx+skippy) < 0 { - return ErrInvalidLengthQuery - } - if (iNdEx + skippy) > l { - return io.ErrUnexpectedEOF - } - iNdEx += skippy - } - } - - if iNdEx > l { - return io.ErrUnexpectedEOF - } - return nil -} func (m *StreamOrderbookUpdate) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/protocol/x/clob/types/streaming.pb.go b/protocol/x/clob/types/streaming.pb.go new file mode 100644 index 0000000000..83b8db719a --- /dev/null +++ b/protocol/x/clob/types/streaming.pb.go @@ -0,0 +1,473 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: dydxprotocol/clob/streaming.proto + +package types + +import ( + fmt "fmt" + proto "github.com/cosmos/gogoproto/proto" + types "github.com/dydxprotocol/v4-chain/protocol/x/subaccounts/types" + io "io" + math "math" + math_bits "math/bits" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package + +// StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. +type StagedFinalizeBlockEvent struct { + // Contains one of StreamOrderbookFill, StreamSubaccountUpdate. + // + // Types that are valid to be assigned to Event: + // *StagedFinalizeBlockEvent_OrderFill + // *StagedFinalizeBlockEvent_SubaccountUpdate + Event isStagedFinalizeBlockEvent_Event `protobuf_oneof:"event"` +} + +func (m *StagedFinalizeBlockEvent) Reset() { *m = StagedFinalizeBlockEvent{} } +func (m *StagedFinalizeBlockEvent) String() string { return proto.CompactTextString(m) } +func (*StagedFinalizeBlockEvent) ProtoMessage() {} +func (*StagedFinalizeBlockEvent) Descriptor() ([]byte, []int) { + return fileDescriptor_cecf6ffcf2554dee, []int{0} +} +func (m *StagedFinalizeBlockEvent) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *StagedFinalizeBlockEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_StagedFinalizeBlockEvent.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *StagedFinalizeBlockEvent) XXX_Merge(src proto.Message) { + xxx_messageInfo_StagedFinalizeBlockEvent.Merge(m, src) +} +func (m *StagedFinalizeBlockEvent) XXX_Size() int { + return m.Size() +} +func (m *StagedFinalizeBlockEvent) XXX_DiscardUnknown() { + xxx_messageInfo_StagedFinalizeBlockEvent.DiscardUnknown(m) +} + +var xxx_messageInfo_StagedFinalizeBlockEvent proto.InternalMessageInfo + +type isStagedFinalizeBlockEvent_Event interface { + isStagedFinalizeBlockEvent_Event() + MarshalTo([]byte) (int, error) + Size() int +} + +type StagedFinalizeBlockEvent_OrderFill struct { + OrderFill *StreamOrderbookFill `protobuf:"bytes,1,opt,name=order_fill,json=orderFill,proto3,oneof" json:"order_fill,omitempty"` +} +type StagedFinalizeBlockEvent_SubaccountUpdate struct { + SubaccountUpdate *types.StreamSubaccountUpdate `protobuf:"bytes,2,opt,name=subaccount_update,json=subaccountUpdate,proto3,oneof" json:"subaccount_update,omitempty"` +} + +func (*StagedFinalizeBlockEvent_OrderFill) isStagedFinalizeBlockEvent_Event() {} +func (*StagedFinalizeBlockEvent_SubaccountUpdate) isStagedFinalizeBlockEvent_Event() {} + +func (m *StagedFinalizeBlockEvent) GetEvent() isStagedFinalizeBlockEvent_Event { + if m != nil { + return m.Event + } + return nil +} + +func (m *StagedFinalizeBlockEvent) GetOrderFill() *StreamOrderbookFill { + if x, ok := m.GetEvent().(*StagedFinalizeBlockEvent_OrderFill); ok { + return x.OrderFill + } + return nil +} + +func (m *StagedFinalizeBlockEvent) GetSubaccountUpdate() *types.StreamSubaccountUpdate { + if x, ok := m.GetEvent().(*StagedFinalizeBlockEvent_SubaccountUpdate); ok { + return x.SubaccountUpdate + } + return nil +} + +// XXX_OneofWrappers is for the internal use of the proto package. +func (*StagedFinalizeBlockEvent) XXX_OneofWrappers() []interface{} { + return []interface{}{ + (*StagedFinalizeBlockEvent_OrderFill)(nil), + (*StagedFinalizeBlockEvent_SubaccountUpdate)(nil), + } +} + +func init() { + proto.RegisterType((*StagedFinalizeBlockEvent)(nil), "dydxprotocol.clob.StagedFinalizeBlockEvent") +} + +func init() { proto.RegisterFile("dydxprotocol/clob/streaming.proto", fileDescriptor_cecf6ffcf2554dee) } + +var fileDescriptor_cecf6ffcf2554dee = []byte{ + // 281 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x4c, 0xa9, 0x4c, 0xa9, + 0x28, 0x28, 0xca, 0x2f, 0xc9, 0x4f, 0xce, 0xcf, 0xd1, 0x4f, 0xce, 0xc9, 0x4f, 0xd2, 0x2f, 0x2e, + 0x29, 0x4a, 0x4d, 0xcc, 0xcd, 0xcc, 0x4b, 0xd7, 0x03, 0x8b, 0x0b, 0x09, 0x22, 0x2b, 0xd1, 0x03, + 0x29, 0x91, 0xd2, 0x40, 0xd1, 0x55, 0x5c, 0x9a, 0x94, 0x98, 0x9c, 0x9c, 0x5f, 0x9a, 0x57, 0x52, + 0x8c, 0xae, 0x59, 0x4a, 0x16, 0xd3, 0xfc, 0xc2, 0xd2, 0xd4, 0xa2, 0x4a, 0x88, 0xb4, 0xd2, 0x59, + 0x46, 0x2e, 0x89, 0xe0, 0x92, 0xc4, 0xf4, 0xd4, 0x14, 0xb7, 0xcc, 0xbc, 0xc4, 0x9c, 0xcc, 0xaa, + 0x54, 0xa7, 0x9c, 0xfc, 0xe4, 0x6c, 0xd7, 0xb2, 0xd4, 0xbc, 0x12, 0x21, 0x77, 0x2e, 0xae, 0xfc, + 0xa2, 0x94, 0xd4, 0xa2, 0xf8, 0xb4, 0xcc, 0x9c, 0x1c, 0x09, 0x46, 0x05, 0x46, 0x0d, 0x6e, 0x23, + 0x35, 0x3d, 0x0c, 0xd7, 0xe8, 0x05, 0x83, 0xed, 0xf4, 0x07, 0x29, 0x4d, 0xca, 0xcf, 0xcf, 0x76, + 0xcb, 0xcc, 0xc9, 0xf1, 0x60, 0x08, 0xe2, 0x04, 0xeb, 0x05, 0x71, 0x84, 0xe2, 0xb9, 0x04, 0x11, + 0x6e, 0x8c, 0x2f, 0x2d, 0x48, 0x49, 0x2c, 0x49, 0x95, 0x60, 0x02, 0x9b, 0x67, 0x80, 0x6a, 0x1e, + 0x92, 0x57, 0xa0, 0xc6, 0x06, 0xc3, 0x45, 0x42, 0xc1, 0xfa, 0x3c, 0x18, 0x82, 0x04, 0x8a, 0xd1, + 0xc4, 0x9c, 0xd8, 0xb9, 0x58, 0x53, 0x41, 0x4e, 0x76, 0x0a, 0x38, 0xf1, 0x48, 0x8e, 0xf1, 0xc2, + 0x23, 0x39, 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96, 0x63, 0xb8, 0xf0, 0x58, 0x8e, 0xe1, + 0xc6, 0x63, 0x39, 0x86, 0x28, 0xb3, 0xf4, 0xcc, 0x92, 0x8c, 0xd2, 0x24, 0xbd, 0xe4, 0xfc, 0x5c, + 0x7d, 0x94, 0x30, 0x29, 0x33, 0xd1, 0x4d, 0xce, 0x48, 0xcc, 0xcc, 0xd3, 0x87, 0x8b, 0x54, 0x40, + 0xc2, 0xa9, 0xa4, 0xb2, 0x20, 0xb5, 0x38, 0x89, 0x0d, 0x2c, 0x6c, 0x0c, 0x08, 0x00, 0x00, 0xff, + 0xff, 0x65, 0x71, 0xd8, 0xa8, 0xa9, 0x01, 0x00, 0x00, +} + +func (m *StagedFinalizeBlockEvent) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *StagedFinalizeBlockEvent) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StagedFinalizeBlockEvent) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Event != nil { + { + size := m.Event.Size() + i -= size + if _, err := m.Event.MarshalTo(dAtA[i:]); err != nil { + return 0, err + } + } + } + return len(dAtA) - i, nil +} + +func (m *StagedFinalizeBlockEvent_OrderFill) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StagedFinalizeBlockEvent_OrderFill) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.OrderFill != nil { + { + size, err := m.OrderFill.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStreaming(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + } + return len(dAtA) - i, nil +} +func (m *StagedFinalizeBlockEvent_SubaccountUpdate) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *StagedFinalizeBlockEvent_SubaccountUpdate) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + if m.SubaccountUpdate != nil { + { + size, err := m.SubaccountUpdate.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintStreaming(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + } + return len(dAtA) - i, nil +} +func encodeVarintStreaming(dAtA []byte, offset int, v uint64) int { + offset -= sovStreaming(v) + base := offset + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return base +} +func (m *StagedFinalizeBlockEvent) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Event != nil { + n += m.Event.Size() + } + return n +} + +func (m *StagedFinalizeBlockEvent_OrderFill) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.OrderFill != nil { + l = m.OrderFill.Size() + n += 1 + l + sovStreaming(uint64(l)) + } + return n +} +func (m *StagedFinalizeBlockEvent_SubaccountUpdate) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.SubaccountUpdate != nil { + l = m.SubaccountUpdate.Size() + n += 1 + l + sovStreaming(uint64(l)) + } + return n +} + +func sovStreaming(x uint64) (n int) { + return (math_bits.Len64(x|1) + 6) / 7 +} +func sozStreaming(x uint64) (n int) { + return sovStreaming(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *StagedFinalizeBlockEvent) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreaming + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: StagedFinalizeBlockEvent: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: StagedFinalizeBlockEvent: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field OrderFill", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreaming + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStreaming + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStreaming + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &StreamOrderbookFill{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Event = &StagedFinalizeBlockEvent_OrderFill{v} + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field SubaccountUpdate", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowStreaming + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthStreaming + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthStreaming + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + v := &types.StreamSubaccountUpdate{} + if err := v.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + m.Event = &StagedFinalizeBlockEvent_SubaccountUpdate{v} + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipStreaming(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthStreaming + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipStreaming(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + depth := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreaming + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreaming + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + case 1: + iNdEx += 8 + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowStreaming + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if length < 0 { + return 0, ErrInvalidLengthStreaming + } + iNdEx += length + case 3: + depth++ + case 4: + if depth == 0 { + return 0, ErrUnexpectedEndOfGroupStreaming + } + depth-- + case 5: + iNdEx += 4 + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + if iNdEx < 0 { + return 0, ErrInvalidLengthStreaming + } + if depth == 0 { + return iNdEx, nil + } + } + return 0, io.ErrUnexpectedEOF +} + +var ( + ErrInvalidLengthStreaming = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowStreaming = fmt.Errorf("proto: integer overflow") + ErrUnexpectedEndOfGroupStreaming = fmt.Errorf("proto: unexpected end of group") +) From ceba0585dda0a8dac96fdfec9e4eeaa8919227b8 Mon Sep 17 00:00:00 2001 From: Teddy Ding Date: Mon, 16 Sep 2024 11:43:31 -0400 Subject: [PATCH 3/5] fix protos --- .../src/codegen/dydxprotocol/bundle.ts | 566 +++++++++--------- .../v4-protos/src/codegen/gogoproto/bundle.ts | 4 +- .../v4-protos/src/codegen/google/bundle.ts | 24 +- 3 files changed, 302 insertions(+), 292 deletions(-) diff --git a/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts b/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts index b12e68826f..3324f5cd90 100644 --- a/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts +++ b/indexer/packages/v4-protos/src/codegen/dydxprotocol/bundle.ts @@ -1,178 +1,186 @@ import * as _5 from "./accountplus/accountplus"; import * as _6 from "./accountplus/genesis"; -import * as _7 from "./affiliates/affiliates"; -import * as _8 from "./affiliates/genesis"; -import * as _9 from "./affiliates/query"; -import * as _10 from "./affiliates/tx"; -import * as _11 from "./assets/asset"; -import * as _12 from "./assets/genesis"; -import * as _13 from "./assets/query"; -import * as _14 from "./assets/tx"; -import * as _15 from "./blocktime/blocktime"; -import * as _16 from "./blocktime/genesis"; -import * as _17 from "./blocktime/params"; -import * as _18 from "./blocktime/query"; -import * as _19 from "./blocktime/tx"; -import * as _20 from "./bridge/bridge_event_info"; -import * as _21 from "./bridge/bridge_event"; -import * as _22 from "./bridge/genesis"; -import * as _23 from "./bridge/params"; -import * as _24 from "./bridge/query"; -import * as _25 from "./bridge/tx"; -import * as _26 from "./clob/block_rate_limit_config"; -import * as _27 from "./clob/clob_pair"; -import * as _28 from "./clob/equity_tier_limit_config"; -import * as _29 from "./clob/genesis"; -import * as _30 from "./clob/liquidations_config"; -import * as _31 from "./clob/liquidations"; -import * as _32 from "./clob/matches"; -import * as _33 from "./clob/mev"; -import * as _34 from "./clob/operation"; -import * as _35 from "./clob/order_removals"; -import * as _36 from "./clob/order"; -import * as _37 from "./clob/process_proposer_matches_events"; -import * as _38 from "./clob/query"; -import * as _39 from "./clob/streaming"; -import * as _40 from "./clob/tx"; -import * as _41 from "./daemons/bridge/bridge"; -import * as _42 from "./daemons/liquidation/liquidation"; -import * as _43 from "./daemons/pricefeed/price_feed"; -import * as _44 from "./delaymsg/block_message_ids"; -import * as _45 from "./delaymsg/delayed_message"; -import * as _46 from "./delaymsg/genesis"; -import * as _47 from "./delaymsg/query"; -import * as _48 from "./delaymsg/tx"; -import * as _49 from "./epochs/epoch_info"; -import * as _50 from "./epochs/genesis"; -import * as _51 from "./epochs/query"; -import * as _52 from "./feetiers/genesis"; -import * as _53 from "./feetiers/params"; -import * as _54 from "./feetiers/query"; -import * as _55 from "./feetiers/tx"; -import * as _56 from "./govplus/genesis"; -import * as _57 from "./govplus/query"; -import * as _58 from "./govplus/tx"; -import * as _59 from "./indexer/events/events"; -import * as _60 from "./indexer/indexer_manager/event"; -import * as _61 from "./indexer/off_chain_updates/off_chain_updates"; -import * as _62 from "./indexer/protocol/v1/clob"; -import * as _63 from "./indexer/protocol/v1/perpetual"; -import * as _64 from "./indexer/protocol/v1/subaccount"; -import * as _65 from "./indexer/redis/redis_order"; -import * as _66 from "./indexer/shared/removal_reason"; -import * as _67 from "./indexer/socks/messages"; -import * as _68 from "./listing/genesis"; -import * as _69 from "./listing/params"; -import * as _70 from "./listing/query"; -import * as _71 from "./listing/tx"; -import * as _72 from "./perpetuals/genesis"; -import * as _73 from "./perpetuals/params"; -import * as _74 from "./perpetuals/perpetual"; -import * as _75 from "./perpetuals/query"; -import * as _76 from "./perpetuals/tx"; -import * as _77 from "./prices/genesis"; -import * as _78 from "./prices/market_param"; -import * as _79 from "./prices/market_price"; -import * as _80 from "./prices/query"; -import * as _81 from "./prices/tx"; -import * as _82 from "./ratelimit/capacity"; -import * as _83 from "./ratelimit/genesis"; -import * as _84 from "./ratelimit/limit_params"; -import * as _85 from "./ratelimit/pending_send_packet"; -import * as _86 from "./ratelimit/query"; -import * as _87 from "./ratelimit/tx"; -import * as _88 from "./revshare/genesis"; -import * as _89 from "./revshare/params"; -import * as _90 from "./revshare/query"; -import * as _91 from "./revshare/revshare"; -import * as _92 from "./revshare/tx"; -import * as _93 from "./rewards/genesis"; -import * as _94 from "./rewards/params"; -import * as _95 from "./rewards/query"; -import * as _96 from "./rewards/reward_share"; -import * as _97 from "./rewards/tx"; -import * as _98 from "./sending/genesis"; -import * as _99 from "./sending/query"; -import * as _100 from "./sending/transfer"; -import * as _101 from "./sending/tx"; -import * as _102 from "./stats/genesis"; -import * as _103 from "./stats/params"; -import * as _104 from "./stats/query"; -import * as _105 from "./stats/stats"; -import * as _106 from "./stats/tx"; -import * as _107 from "./subaccounts/asset_position"; -import * as _108 from "./subaccounts/genesis"; -import * as _109 from "./subaccounts/perpetual_position"; -import * as _110 from "./subaccounts/query"; -import * as _111 from "./subaccounts/streaming"; -import * as _112 from "./subaccounts/subaccount"; -import * as _113 from "./vault/genesis"; -import * as _114 from "./vault/params"; -import * as _115 from "./vault/query"; -import * as _116 from "./vault/share"; -import * as _117 from "./vault/tx"; -import * as _118 from "./vault/vault"; -import * as _119 from "./vest/genesis"; -import * as _120 from "./vest/query"; -import * as _121 from "./vest/tx"; -import * as _122 from "./vest/vest_entry"; -import * as _130 from "./assets/query.lcd"; -import * as _131 from "./blocktime/query.lcd"; -import * as _132 from "./bridge/query.lcd"; -import * as _133 from "./clob/query.lcd"; -import * as _134 from "./delaymsg/query.lcd"; -import * as _135 from "./epochs/query.lcd"; -import * as _136 from "./feetiers/query.lcd"; -import * as _137 from "./listing/query.lcd"; -import * as _138 from "./perpetuals/query.lcd"; -import * as _139 from "./prices/query.lcd"; -import * as _140 from "./ratelimit/query.lcd"; -import * as _141 from "./revshare/query.lcd"; -import * as _142 from "./rewards/query.lcd"; -import * as _143 from "./stats/query.lcd"; -import * as _144 from "./subaccounts/query.lcd"; -import * as _145 from "./vault/query.lcd"; -import * as _146 from "./vest/query.lcd"; -import * as _147 from "./affiliates/query.rpc.Query"; -import * as _148 from "./assets/query.rpc.Query"; -import * as _149 from "./blocktime/query.rpc.Query"; -import * as _150 from "./bridge/query.rpc.Query"; -import * as _151 from "./clob/query.rpc.Query"; -import * as _152 from "./delaymsg/query.rpc.Query"; -import * as _153 from "./epochs/query.rpc.Query"; -import * as _154 from "./feetiers/query.rpc.Query"; -import * as _155 from "./govplus/query.rpc.Query"; -import * as _156 from "./listing/query.rpc.Query"; -import * as _157 from "./perpetuals/query.rpc.Query"; -import * as _158 from "./prices/query.rpc.Query"; -import * as _159 from "./ratelimit/query.rpc.Query"; -import * as _160 from "./revshare/query.rpc.Query"; -import * as _161 from "./rewards/query.rpc.Query"; -import * as _162 from "./sending/query.rpc.Query"; -import * as _163 from "./stats/query.rpc.Query"; -import * as _164 from "./subaccounts/query.rpc.Query"; -import * as _165 from "./vault/query.rpc.Query"; -import * as _166 from "./vest/query.rpc.Query"; -import * as _167 from "./affiliates/tx.rpc.msg"; -import * as _168 from "./blocktime/tx.rpc.msg"; -import * as _169 from "./bridge/tx.rpc.msg"; -import * as _170 from "./clob/tx.rpc.msg"; -import * as _171 from "./delaymsg/tx.rpc.msg"; -import * as _172 from "./feetiers/tx.rpc.msg"; -import * as _173 from "./govplus/tx.rpc.msg"; -import * as _174 from "./listing/tx.rpc.msg"; -import * as _175 from "./perpetuals/tx.rpc.msg"; -import * as _176 from "./prices/tx.rpc.msg"; -import * as _177 from "./ratelimit/tx.rpc.msg"; -import * as _178 from "./revshare/tx.rpc.msg"; -import * as _179 from "./rewards/tx.rpc.msg"; -import * as _180 from "./sending/tx.rpc.msg"; -import * as _181 from "./stats/tx.rpc.msg"; -import * as _182 from "./vault/tx.rpc.msg"; -import * as _183 from "./vest/tx.rpc.msg"; -import * as _184 from "./lcd"; -import * as _185 from "./rpc.query"; -import * as _186 from "./rpc.tx"; +import * as _7 from "./accountplus/models"; +import * as _8 from "./accountplus/params"; +import * as _9 from "./accountplus/query"; +import * as _10 from "./accountplus/tx"; +import * as _11 from "./affiliates/affiliates"; +import * as _12 from "./affiliates/genesis"; +import * as _13 from "./affiliates/query"; +import * as _14 from "./affiliates/tx"; +import * as _15 from "./assets/asset"; +import * as _16 from "./assets/genesis"; +import * as _17 from "./assets/query"; +import * as _18 from "./assets/tx"; +import * as _19 from "./blocktime/blocktime"; +import * as _20 from "./blocktime/genesis"; +import * as _21 from "./blocktime/params"; +import * as _22 from "./blocktime/query"; +import * as _23 from "./blocktime/tx"; +import * as _24 from "./bridge/bridge_event_info"; +import * as _25 from "./bridge/bridge_event"; +import * as _26 from "./bridge/genesis"; +import * as _27 from "./bridge/params"; +import * as _28 from "./bridge/query"; +import * as _29 from "./bridge/tx"; +import * as _30 from "./clob/block_rate_limit_config"; +import * as _31 from "./clob/clob_pair"; +import * as _32 from "./clob/equity_tier_limit_config"; +import * as _33 from "./clob/genesis"; +import * as _34 from "./clob/liquidations_config"; +import * as _35 from "./clob/liquidations"; +import * as _36 from "./clob/matches"; +import * as _37 from "./clob/mev"; +import * as _38 from "./clob/operation"; +import * as _39 from "./clob/order_removals"; +import * as _40 from "./clob/order"; +import * as _41 from "./clob/process_proposer_matches_events"; +import * as _42 from "./clob/query"; +import * as _43 from "./clob/streaming"; +import * as _44 from "./clob/tx"; +import * as _45 from "./daemons/bridge/bridge"; +import * as _46 from "./daemons/liquidation/liquidation"; +import * as _47 from "./daemons/pricefeed/price_feed"; +import * as _48 from "./delaymsg/block_message_ids"; +import * as _49 from "./delaymsg/delayed_message"; +import * as _50 from "./delaymsg/genesis"; +import * as _51 from "./delaymsg/query"; +import * as _52 from "./delaymsg/tx"; +import * as _53 from "./epochs/epoch_info"; +import * as _54 from "./epochs/genesis"; +import * as _55 from "./epochs/query"; +import * as _56 from "./feetiers/genesis"; +import * as _57 from "./feetiers/params"; +import * as _58 from "./feetiers/query"; +import * as _59 from "./feetiers/tx"; +import * as _60 from "./govplus/genesis"; +import * as _61 from "./govplus/query"; +import * as _62 from "./govplus/tx"; +import * as _63 from "./indexer/events/events"; +import * as _64 from "./indexer/indexer_manager/event"; +import * as _65 from "./indexer/off_chain_updates/off_chain_updates"; +import * as _66 from "./indexer/protocol/v1/clob"; +import * as _67 from "./indexer/protocol/v1/perpetual"; +import * as _68 from "./indexer/protocol/v1/subaccount"; +import * as _69 from "./indexer/protocol/v1/vault"; +import * as _70 from "./indexer/redis/redis_order"; +import * as _71 from "./indexer/shared/removal_reason"; +import * as _72 from "./indexer/socks/messages"; +import * as _73 from "./listing/genesis"; +import * as _74 from "./listing/params"; +import * as _75 from "./listing/query"; +import * as _76 from "./listing/tx"; +import * as _77 from "./perpetuals/genesis"; +import * as _78 from "./perpetuals/params"; +import * as _79 from "./perpetuals/perpetual"; +import * as _80 from "./perpetuals/query"; +import * as _81 from "./perpetuals/tx"; +import * as _82 from "./prices/genesis"; +import * as _83 from "./prices/market_param"; +import * as _84 from "./prices/market_price"; +import * as _85 from "./prices/query"; +import * as _86 from "./prices/tx"; +import * as _87 from "./ratelimit/capacity"; +import * as _88 from "./ratelimit/genesis"; +import * as _89 from "./ratelimit/limit_params"; +import * as _90 from "./ratelimit/pending_send_packet"; +import * as _91 from "./ratelimit/query"; +import * as _92 from "./ratelimit/tx"; +import * as _93 from "./revshare/genesis"; +import * as _94 from "./revshare/params"; +import * as _95 from "./revshare/query"; +import * as _96 from "./revshare/revshare"; +import * as _97 from "./revshare/tx"; +import * as _98 from "./rewards/genesis"; +import * as _99 from "./rewards/params"; +import * as _100 from "./rewards/query"; +import * as _101 from "./rewards/reward_share"; +import * as _102 from "./rewards/tx"; +import * as _103 from "./sending/genesis"; +import * as _104 from "./sending/query"; +import * as _105 from "./sending/transfer"; +import * as _106 from "./sending/tx"; +import * as _107 from "./stats/genesis"; +import * as _108 from "./stats/params"; +import * as _109 from "./stats/query"; +import * as _110 from "./stats/stats"; +import * as _111 from "./stats/tx"; +import * as _112 from "./subaccounts/asset_position"; +import * as _113 from "./subaccounts/genesis"; +import * as _114 from "./subaccounts/perpetual_position"; +import * as _115 from "./subaccounts/query"; +import * as _116 from "./subaccounts/streaming"; +import * as _117 from "./subaccounts/subaccount"; +import * as _118 from "./vault/genesis"; +import * as _119 from "./vault/params"; +import * as _120 from "./vault/query"; +import * as _121 from "./vault/share"; +import * as _122 from "./vault/tx"; +import * as _123 from "./vault/vault"; +import * as _124 from "./vest/genesis"; +import * as _125 from "./vest/query"; +import * as _126 from "./vest/tx"; +import * as _127 from "./vest/vest_entry"; +import * as _135 from "./accountplus/query.lcd"; +import * as _136 from "./assets/query.lcd"; +import * as _137 from "./blocktime/query.lcd"; +import * as _138 from "./bridge/query.lcd"; +import * as _139 from "./clob/query.lcd"; +import * as _140 from "./delaymsg/query.lcd"; +import * as _141 from "./epochs/query.lcd"; +import * as _142 from "./feetiers/query.lcd"; +import * as _143 from "./listing/query.lcd"; +import * as _144 from "./perpetuals/query.lcd"; +import * as _145 from "./prices/query.lcd"; +import * as _146 from "./ratelimit/query.lcd"; +import * as _147 from "./revshare/query.lcd"; +import * as _148 from "./rewards/query.lcd"; +import * as _149 from "./stats/query.lcd"; +import * as _150 from "./subaccounts/query.lcd"; +import * as _151 from "./vault/query.lcd"; +import * as _152 from "./vest/query.lcd"; +import * as _153 from "./accountplus/query.rpc.Query"; +import * as _154 from "./affiliates/query.rpc.Query"; +import * as _155 from "./assets/query.rpc.Query"; +import * as _156 from "./blocktime/query.rpc.Query"; +import * as _157 from "./bridge/query.rpc.Query"; +import * as _158 from "./clob/query.rpc.Query"; +import * as _159 from "./delaymsg/query.rpc.Query"; +import * as _160 from "./epochs/query.rpc.Query"; +import * as _161 from "./feetiers/query.rpc.Query"; +import * as _162 from "./govplus/query.rpc.Query"; +import * as _163 from "./listing/query.rpc.Query"; +import * as _164 from "./perpetuals/query.rpc.Query"; +import * as _165 from "./prices/query.rpc.Query"; +import * as _166 from "./ratelimit/query.rpc.Query"; +import * as _167 from "./revshare/query.rpc.Query"; +import * as _168 from "./rewards/query.rpc.Query"; +import * as _169 from "./sending/query.rpc.Query"; +import * as _170 from "./stats/query.rpc.Query"; +import * as _171 from "./subaccounts/query.rpc.Query"; +import * as _172 from "./vault/query.rpc.Query"; +import * as _173 from "./vest/query.rpc.Query"; +import * as _174 from "./accountplus/tx.rpc.msg"; +import * as _175 from "./affiliates/tx.rpc.msg"; +import * as _176 from "./blocktime/tx.rpc.msg"; +import * as _177 from "./bridge/tx.rpc.msg"; +import * as _178 from "./clob/tx.rpc.msg"; +import * as _179 from "./delaymsg/tx.rpc.msg"; +import * as _180 from "./feetiers/tx.rpc.msg"; +import * as _181 from "./govplus/tx.rpc.msg"; +import * as _182 from "./listing/tx.rpc.msg"; +import * as _183 from "./perpetuals/tx.rpc.msg"; +import * as _184 from "./prices/tx.rpc.msg"; +import * as _185 from "./ratelimit/tx.rpc.msg"; +import * as _186 from "./revshare/tx.rpc.msg"; +import * as _187 from "./rewards/tx.rpc.msg"; +import * as _188 from "./sending/tx.rpc.msg"; +import * as _189 from "./stats/tx.rpc.msg"; +import * as _190 from "./vault/tx.rpc.msg"; +import * as _191 from "./vest/tx.rpc.msg"; +import * as _192 from "./lcd"; +import * as _193 from "./rpc.query"; +import * as _194 from "./rpc.tx"; export namespace dydxprotocol { export const accountplus = { ..._5, ..._6, @@ -180,35 +188,32 @@ export namespace dydxprotocol { ..._8, ..._9, ..._10, - ..._147, - ..._167 + ..._135, + ..._153, + ..._174 }; export const affiliates = { ..._11, ..._12, ..._13, ..._14, - ..._130, - ..._148 + ..._154, + ..._175 }; export const assets = { ..._15, ..._16, ..._17, ..._18, - ..._19, - ..._131, - ..._149, - ..._168 + ..._136, + ..._155 }; export const blocktime = { ..._19, ..._20, ..._21, ..._22, ..._23, - ..._24, - ..._25, - ..._132, - ..._150, - ..._169 + ..._137, + ..._156, + ..._176 }; export const bridge = { ..._24, ..._25, @@ -216,9 +221,9 @@ export namespace dydxprotocol { ..._27, ..._28, ..._29, - ..._137, - ..._156, - ..._176 + ..._138, + ..._157, + ..._177 }; export const clob = { ..._30, ..._31, @@ -231,166 +236,171 @@ export namespace dydxprotocol { ..._38, ..._39, ..._40, - ..._133, - ..._151, - ..._170 + ..._41, + ..._42, + ..._43, + ..._44, + ..._139, + ..._158, + ..._178 }; export namespace daemons { - export const bridge = { ..._41 + export const bridge = { ..._45 }; - export const liquidation = { ..._42 + export const liquidation = { ..._46 }; - export const pricefeed = { ..._43 + export const pricefeed = { ..._47 }; } - export const delaymsg = { ..._44, - ..._45, - ..._46, - ..._47, - ..._48, - ..._134, - ..._152, - ..._171 - }; - export const epochs = { ..._49, + export const delaymsg = { ..._48, + ..._49, ..._50, ..._51, - ..._135, - ..._153 + ..._52, + ..._140, + ..._159, + ..._179 }; - export const feetiers = { ..._52, - ..._53, + export const epochs = { ..._53, ..._54, ..._55, - ..._136, - ..._154, - ..._172 + ..._141, + ..._160 }; - export const govplus = { ..._56, + export const feetiers = { ..._56, ..._57, ..._58, - ..._155, - ..._173 + ..._59, + ..._142, + ..._161, + ..._180 + }; + export const govplus = { ..._60, + ..._61, + ..._62, + ..._162, + ..._181 }; export namespace indexer { - export const events = { ..._59 + export const events = { ..._63 }; - export const indexer_manager = { ..._60 + export const indexer_manager = { ..._64 }; - export const off_chain_updates = { ..._61 + export const off_chain_updates = { ..._65 }; export namespace protocol { - export const v1 = { ..._62, - ..._63, - ..._64 + export const v1 = { ..._66, + ..._67, + ..._68, + ..._69 }; } - export const redis = { ..._65 + export const redis = { ..._70 }; - export const shared = { ..._66 + export const shared = { ..._71 }; - export const socks = { ..._67 + export const socks = { ..._72 }; } - export const listing = { ..._68, - ..._69, - ..._70, - ..._71, - ..._137, - ..._156, - ..._174 - }; - export const perpetuals = { ..._72, - ..._73, + export const listing = { ..._73, ..._74, ..._75, ..._76, - ..._138, - ..._157, - ..._175 + ..._143, + ..._163, + ..._182 }; - export const prices = { ..._77, + export const perpetuals = { ..._77, ..._78, ..._79, ..._80, ..._81, - ..._139, - ..._158, - ..._176 + ..._144, + ..._164, + ..._183 }; - export const ratelimit = { ..._82, + export const prices = { ..._82, ..._83, ..._84, ..._85, ..._86, - ..._87, - ..._140, - ..._159, - ..._177 + ..._145, + ..._165, + ..._184 }; - export const revshare = { ..._88, + export const ratelimit = { ..._87, + ..._88, ..._89, ..._90, ..._91, ..._92, - ..._141, - ..._160, - ..._178 + ..._146, + ..._166, + ..._185 }; - export const rewards = { ..._93, + export const revshare = { ..._93, ..._94, ..._95, ..._96, ..._97, - ..._142, - ..._161, - ..._179 + ..._147, + ..._167, + ..._186 }; - export const sending = { ..._98, + export const rewards = { ..._98, ..._99, ..._100, ..._101, - ..._162, - ..._180 + ..._102, + ..._148, + ..._168, + ..._187 }; - export const stats = { ..._102, - ..._103, + export const sending = { ..._103, ..._104, ..._105, ..._106, - ..._143, - ..._163, - ..._181 + ..._169, + ..._188 }; - export const subaccounts = { ..._107, + export const stats = { ..._107, ..._108, ..._109, ..._110, ..._111, - ..._112, - ..._144, - ..._164 + ..._149, + ..._170, + ..._189 }; - export const vault = { ..._113, + export const subaccounts = { ..._112, + ..._113, ..._114, ..._115, ..._116, ..._117, - ..._118, - ..._145, - ..._165, - ..._182 + ..._150, + ..._171 }; - export const vest = { ..._119, + export const vault = { ..._118, + ..._119, ..._120, ..._121, ..._122, - ..._146, - ..._166, - ..._183 + ..._123, + ..._151, + ..._172, + ..._190 }; - export const ClientFactory = { ..._184, - ..._185, - ..._186 + export const vest = { ..._124, + ..._125, + ..._126, + ..._127, + ..._152, + ..._173, + ..._191 + }; + export const ClientFactory = { ..._192, + ..._193, + ..._194 }; } \ No newline at end of file diff --git a/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts b/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts index 01ed58731a..00375897ff 100644 --- a/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts +++ b/indexer/packages/v4-protos/src/codegen/gogoproto/bundle.ts @@ -1,3 +1,3 @@ -import * as _123 from "./gogo"; -export const gogoproto = { ..._123 +import * as _128 from "./gogo"; +export const gogoproto = { ..._128 }; \ No newline at end of file diff --git a/indexer/packages/v4-protos/src/codegen/google/bundle.ts b/indexer/packages/v4-protos/src/codegen/google/bundle.ts index a2768945ac..b2f572879c 100644 --- a/indexer/packages/v4-protos/src/codegen/google/bundle.ts +++ b/indexer/packages/v4-protos/src/codegen/google/bundle.ts @@ -1,16 +1,16 @@ -import * as _124 from "./api/annotations"; -import * as _125 from "./api/http"; -import * as _126 from "./protobuf/descriptor"; -import * as _127 from "./protobuf/duration"; -import * as _128 from "./protobuf/timestamp"; -import * as _129 from "./protobuf/any"; +import * as _129 from "./api/annotations"; +import * as _130 from "./api/http"; +import * as _131 from "./protobuf/descriptor"; +import * as _132 from "./protobuf/duration"; +import * as _133 from "./protobuf/timestamp"; +import * as _134 from "./protobuf/any"; export namespace google { - export const api = { ..._124, - ..._125 + export const api = { ..._129, + ..._130 }; - export const protobuf = { ..._126, - ..._127, - ..._128, - ..._129 + export const protobuf = { ..._131, + ..._132, + ..._133, + ..._134 }; } \ No newline at end of file From 4f9614093da60c1d9db0d7af31a6fac7bbb2d143 Mon Sep 17 00:00:00 2001 From: Teddy Ding Date: Mon, 16 Sep 2024 11:49:18 -0400 Subject: [PATCH 4/5] Add missing codegen file --- .../codegen/dydxprotocol/clob/streaming.ts | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/streaming.ts diff --git a/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/streaming.ts b/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/streaming.ts new file mode 100644 index 0000000000..dab8f1e122 --- /dev/null +++ b/indexer/packages/v4-protos/src/codegen/dydxprotocol/clob/streaming.ts @@ -0,0 +1,71 @@ +import { StreamOrderbookFill, StreamOrderbookFillSDKType } from "./query"; +import { StreamSubaccountUpdate, StreamSubaccountUpdateSDKType } from "../subaccounts/streaming"; +import * as _m0 from "protobufjs/minimal"; +import { DeepPartial } from "../../helpers"; +/** StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. */ + +export interface StagedFinalizeBlockEvent { + orderFill?: StreamOrderbookFill; + subaccountUpdate?: StreamSubaccountUpdate; +} +/** StagedFinalizeBlockEvent is an event staged during `FinalizeBlock`. */ + +export interface StagedFinalizeBlockEventSDKType { + order_fill?: StreamOrderbookFillSDKType; + subaccount_update?: StreamSubaccountUpdateSDKType; +} + +function createBaseStagedFinalizeBlockEvent(): StagedFinalizeBlockEvent { + return { + orderFill: undefined, + subaccountUpdate: undefined + }; +} + +export const StagedFinalizeBlockEvent = { + encode(message: StagedFinalizeBlockEvent, writer: _m0.Writer = _m0.Writer.create()): _m0.Writer { + if (message.orderFill !== undefined) { + StreamOrderbookFill.encode(message.orderFill, writer.uint32(10).fork()).ldelim(); + } + + if (message.subaccountUpdate !== undefined) { + StreamSubaccountUpdate.encode(message.subaccountUpdate, writer.uint32(18).fork()).ldelim(); + } + + return writer; + }, + + decode(input: _m0.Reader | Uint8Array, length?: number): StagedFinalizeBlockEvent { + const reader = input instanceof _m0.Reader ? input : new _m0.Reader(input); + let end = length === undefined ? reader.len : reader.pos + length; + const message = createBaseStagedFinalizeBlockEvent(); + + while (reader.pos < end) { + const tag = reader.uint32(); + + switch (tag >>> 3) { + case 1: + message.orderFill = StreamOrderbookFill.decode(reader, reader.uint32()); + break; + + case 2: + message.subaccountUpdate = StreamSubaccountUpdate.decode(reader, reader.uint32()); + break; + + default: + reader.skipType(tag & 7); + break; + } + } + + return message; + }, + + fromPartial(object: DeepPartial): StagedFinalizeBlockEvent { + const message = createBaseStagedFinalizeBlockEvent(); + message.orderFill = object.orderFill !== undefined && object.orderFill !== null ? StreamOrderbookFill.fromPartial(object.orderFill) : undefined; + message.subaccountUpdate = object.subaccountUpdate !== undefined && object.subaccountUpdate !== null ? StreamSubaccountUpdate.fromPartial(object.subaccountUpdate) : undefined; + return message; + } + +}; \ No newline at end of file From 840bbd7bd8dc9411c77fb8b196aa36710c40b4a3 Mon Sep 17 00:00:00 2001 From: Teddy Ding Date: Mon, 16 Sep 2024 11:57:47 -0400 Subject: [PATCH 5/5] fix lint --- protocol/streaming/full_node_streaming_manager.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/protocol/streaming/full_node_streaming_manager.go b/protocol/streaming/full_node_streaming_manager.go index 0bf1f91476..34b7eaaf36 100644 --- a/protocol/streaming/full_node_streaming_manager.go +++ b/protocol/streaming/full_node_streaming_manager.go @@ -403,8 +403,8 @@ func (sm *FullNodeStreamingManagerImpl) StageFinalizeBlockSubaccountUpdate( } // Stage a fill event in transient store, during `FinalizeBlock`. -// Since `FinalizeBlock` code block can be called more than once with optimistc -// execution (once optimistcally and optionally once on the canonical block), +// Since `FinalizeBlock` code block can be called more than once with optimistic +// execution (once optimistically and optionally once on the canonical block), // we need to stage the events in transient store and later emit them // during `Precommit`. func (sm *FullNodeStreamingManagerImpl) StageFinalizeBlockFill(