From e68a9b2f1dc7edbd9d7d6a67bbfb1fa68f2519e5 Mon Sep 17 00:00:00 2001 From: Sam Calder-Mason Date: Wed, 6 Sep 2023 12:56:26 +1000 Subject: [PATCH] feat(cannon): Connect to Coordinator (#179) * feat(coordinator): Add new Cannon types * feat: Split out derivers and hook up coordinator rpc * feat: Split out derivers and hook up coordinator rpc * refactor: set maximum interval for backoff to 1 minute --- pkg/cannon/cannon.go | 201 +++-- pkg/cannon/config.go | 18 +- pkg/cannon/coordinator/client.go | 106 +++ pkg/cannon/coordinator/config.go | 19 + .../beacon/eth/v2/attester_slashing.go | 250 ++++++ .../deriver/beacon/eth/v2/beacon_block.go | 44 + .../beacon/eth/v2/bls_to_execution_change.go | 242 ++++++ pkg/cannon/deriver/beacon/eth/v2/deposit.go | 250 ++++++ .../beacon/eth/v2/execution_transaction.go | 290 +++++++ .../beacon/eth/v2/proposer_slashing.go | 247 ++++++ .../deriver/beacon/eth/v2/voluntary_exit.go | 243 ++++++ pkg/cannon/deriver/config.go | 16 + pkg/cannon/deriver/event_deriver.go | 26 + pkg/cannon/ethereum/beacon.go | 43 +- pkg/cannon/ethereum/config.go | 10 +- .../event/beacon/eth/v2/attester_slashing.go | 123 --- .../event/beacon/eth/v2/beacon_block.go | 129 --- .../beacon/eth/v2/bls_to_execution_change.go | 116 --- pkg/cannon/event/beacon/eth/v2/deposit.go | 127 --- .../event/beacon/eth/v2/event_deriver.go | 13 - .../beacon/eth/v2/execution_transaction.go | 163 ---- .../event/beacon/eth/v2/proposer_slashing.go | 123 --- .../event/beacon/eth/v2/voluntary_exit.go | 120 --- pkg/cannon/iterator/iterator.go | 17 + pkg/cannon/iterator/slot_iterator.go | 162 ++++ pkg/cannon/metrics.go | 24 +- pkg/networks/network.go | 2 + pkg/proto/xatu/coordinator.pb.go | 564 +++++++++--- pkg/proto/xatu/coordinator.proto | 18 + pkg/proto/xatu/event_ingester.pb.go | 818 +++++++++--------- pkg/proto/xatu/event_ingester.proto | 4 + pkg/server/persistence/cannon/location.go | 76 ++ 32 files changed, 3074 insertions(+), 1530 deletions(-) create mode 100644 pkg/cannon/coordinator/client.go create mode 100644 pkg/cannon/coordinator/config.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/attester_slashing.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/beacon_block.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/bls_to_execution_change.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/deposit.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/execution_transaction.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/proposer_slashing.go create mode 100644 pkg/cannon/deriver/beacon/eth/v2/voluntary_exit.go create mode 100644 pkg/cannon/deriver/config.go create mode 100644 pkg/cannon/deriver/event_deriver.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/attester_slashing.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/beacon_block.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/bls_to_execution_change.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/deposit.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/event_deriver.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/execution_transaction.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/proposer_slashing.go delete mode 100644 pkg/cannon/event/beacon/eth/v2/voluntary_exit.go create mode 100644 pkg/cannon/iterator/iterator.go create mode 100644 pkg/cannon/iterator/slot_iterator.go diff --git a/pkg/cannon/cannon.go b/pkg/cannon/cannon.go index 2eaec2ba..fb424ae7 100644 --- a/pkg/cannon/cannon.go +++ b/pkg/cannon/cannon.go @@ -15,8 +15,11 @@ import ( _ "net/http/pprof" "github.com/beevik/ntp" + "github.com/ethpandaops/xatu/pkg/cannon/coordinator" + "github.com/ethpandaops/xatu/pkg/cannon/deriver" + v2 "github.com/ethpandaops/xatu/pkg/cannon/deriver/beacon/eth/v2" "github.com/ethpandaops/xatu/pkg/cannon/ethereum" - v2 "github.com/ethpandaops/xatu/pkg/cannon/event/beacon/eth/v2" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" "github.com/ethpandaops/xatu/pkg/output" "github.com/ethpandaops/xatu/pkg/proto/xatu" "github.com/go-co-op/gocron" @@ -42,7 +45,9 @@ type Cannon struct { scheduler *gocron.Scheduler - beaconBlockDerivers []v2.BeaconBlockEventDeriver + eventDerivers []deriver.EventDeriver + + coordinatorClient *coordinator.Client } func New(ctx context.Context, log logrus.FieldLogger, config *Config) (*Cannon, error) { @@ -64,25 +69,22 @@ func New(ctx context.Context, log logrus.FieldLogger, config *Config) (*Cannon, return nil, err } - beaconBlockDerivers := []v2.BeaconBlockEventDeriver{ - v2.NewAttesterSlashingDeriver(log), - v2.NewProposerSlashingDeriver(log), - v2.NewVoluntaryExitDeriver(log), - v2.NewDepositDeriver(log), - v2.NewBLSToExecutionChangeDeriver(log), - v2.NewExecutionTransactionDeriver(log), + coordinatorClient, err := coordinator.New(&config.Coordinator, log) + if err != nil { + return nil, err } return &Cannon{ - Config: config, - sinks: sinks, - beacon: beacon, - clockDrift: time.Duration(0), - log: log, - id: uuid.New(), - metrics: NewMetrics("xatu_cannon"), - scheduler: gocron.NewScheduler(time.Local), - beaconBlockDerivers: beaconBlockDerivers, + Config: config, + sinks: sinks, + beacon: beacon, + clockDrift: time.Duration(0), + log: log, + id: uuid.New(), + metrics: NewMetrics("xatu_cannon"), + scheduler: gocron.NewScheduler(time.Local), + eventDerivers: nil, // Derivers are created once the beacon node is ready + coordinatorClient: coordinatorClient, }, nil } @@ -249,20 +251,6 @@ func (c *Cannon) handleNewDecoratedEvent(ctx context.Context, event *xatu.Decora return err } - network := event.GetMeta().GetClient().GetEthereum().GetNetwork().GetId() - networkStr := fmt.Sprintf("%d", network) - - if networkStr == "" || networkStr == "0" { - networkStr = "unknown" - } - - eventType := event.GetEvent().GetName().String() - if eventType == "" { - eventType = "unknown" - } - - c.metrics.AddDecoratedEvent(1, eventType, networkStr) - for _, sink := range c.sinks { if err := sink.HandleNewDecoratedEvent(ctx, event); err != nil { c.log. @@ -278,59 +266,124 @@ func (c *Cannon) handleNewDecoratedEvent(ctx context.Context, event *xatu.Decora func (c *Cannon) startBeaconBlockProcessor(ctx context.Context) error { c.beacon.OnReady(ctx, func(ctx context.Context) error { - c.log.Info("Internal beacon node is ready, firing up beacon block processor") + c.log.Info("Internal beacon node is ready, firing up event derivers") - // TODO: Fetch our starting point from xatu-server - start := uint64(5193791) + networkID := fmt.Sprintf("%d", c.beacon.Metadata().Network.ID) + wallclock := c.beacon.Metadata().Wallclock() - for { - select { - case <-ctx.Done(): - return nil - default: - time.Sleep(1 * time.Second) - - c.log.WithField("slot", start).Info("Processing beacon block") - - if err := c.processBeaconBlock(ctx, start); err != nil { - c.log.WithError(err).Error("Failed to process beacon block") - - // TODO: Make sure we don't tell xatu-server we've processed this block - // If we can't process it, we should stop here and wait for - // human intervention. - } + clientMeta, err := c.createNewClientMeta(ctx) + if err != nil { + return err + } - start++ - } + eventDerivers := []deriver.EventDeriver{ + v2.NewAttesterSlashingDeriver( + c.log, + &c.Config.Derivers.AttesterSlashingConfig, + iterator.NewSlotIterator( + c.log, + networkID, + xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING, + c.coordinatorClient, + wallclock, + ), + c.beacon, + clientMeta, + ), + v2.NewProposerSlashingDeriver( + c.log, + &c.Config.Derivers.ProposerSlashingConfig, + iterator.NewSlotIterator( + c.log, + networkID, + xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING, + c.coordinatorClient, + wallclock, + ), + c.beacon, + clientMeta, + ), + v2.NewVoluntaryExitDeriver( + c.log, + &c.Config.Derivers.VoluntaryExitConfig, + iterator.NewSlotIterator( + c.log, + networkID, + xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT, + c.coordinatorClient, + wallclock, + ), + c.beacon, + clientMeta, + ), + v2.NewDepositDeriver( + c.log, + &c.Config.Derivers.DepositConfig, + iterator.NewSlotIterator( + c.log, + networkID, + xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT, + c.coordinatorClient, + wallclock, + ), + c.beacon, + clientMeta, + ), + v2.NewBLSToExecutionChangeDeriver( + c.log, + &c.Config.Derivers.BLSToExecutionConfig, + iterator.NewSlotIterator( + c.log, + networkID, + xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE, + c.coordinatorClient, + wallclock, + ), + c.beacon, + clientMeta, + ), + v2.NewExecutionTransactionDeriver( + c.log, + &c.Config.Derivers.ExecutionTransactionConfig, + iterator.NewSlotIterator( + c.log, + networkID, + xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION, + c.coordinatorClient, + wallclock, + ), + c.beacon, + clientMeta, + ), } - }) - return nil -} + c.eventDerivers = eventDerivers -func (c *Cannon) processBeaconBlock(ctx context.Context, slot uint64) error { - block, err := c.beacon.Node().FetchBlock(ctx, fmt.Sprintf("%d", slot)) - if err != nil { - return err - } + for _, deriver := range c.eventDerivers { + d := deriver - meta, err := c.createNewClientMeta(ctx) - if err != nil { - return err - } + d.OnEventDerived(ctx, func(ctx context.Context, event *xatu.DecoratedEvent) error { + return c.handleNewDecoratedEvent(ctx, event) + }) - event := v2.NewBeaconBlockMetadata(c.log, block, time.Now(), c.beacon, meta, c.beaconBlockDerivers) + d.OnLocationUpdated(ctx, func(ctx context.Context, location uint64) error { + c.metrics.SetDeriverLocation(location, d.CannonType()) - events, err := event.Process(ctx) - if err != nil { - return err - } + return nil + }) - for _, event := range events { - if err := c.handleNewDecoratedEvent(ctx, event); err != nil { - return err + c.log. + WithField("deriver", deriver.Name()). + WithField("type", deriver.CannonType()). + Info("Starting cannon event deriver") + + if err := deriver.Start(ctx); err != nil { + return err + } } - } + + return nil + }) return nil } diff --git a/pkg/cannon/config.go b/pkg/cannon/config.go index bb9d1854..df984ed4 100644 --- a/pkg/cannon/config.go +++ b/pkg/cannon/config.go @@ -4,6 +4,8 @@ import ( "errors" "fmt" + "github.com/ethpandaops/xatu/pkg/cannon/coordinator" + "github.com/ethpandaops/xatu/pkg/cannon/deriver" "github.com/ethpandaops/xatu/pkg/cannon/ethereum" "github.com/ethpandaops/xatu/pkg/output" "github.com/sirupsen/logrus" @@ -28,6 +30,12 @@ type Config struct { // NTP Server to use for clock drift correction NTPServer string `yaml:"ntpServer" default:"time.google.com"` + + // Derivers configures the cannon with event derivers + Derivers deriver.Config `yaml:"derivers"` + + // Coordinator configuration + Coordinator coordinator.Config `yaml:"coordinator"` } func (c *Config) Validate() error { @@ -41,10 +49,18 @@ func (c *Config) Validate() error { for _, output := range c.Outputs { if err := output.Validate(); err != nil { - return fmt.Errorf("output %s: %w", output.Name, err) + return fmt.Errorf("invalid output config %s: %w", output.Name, err) } } + if err := c.Derivers.Validate(); err != nil { + return fmt.Errorf("invalid derivers config: %w", err) + } + + if err := c.Coordinator.Validate(); err != nil { + return fmt.Errorf("invalid coordinator config: %w", err) + } + return nil } diff --git a/pkg/cannon/coordinator/client.go b/pkg/cannon/coordinator/client.go new file mode 100644 index 00000000..d3113042 --- /dev/null +++ b/pkg/cannon/coordinator/client.go @@ -0,0 +1,106 @@ +package coordinator + +import ( + "context" + "errors" + "fmt" + "net" + + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/sirupsen/logrus" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/encoding/gzip" + "google.golang.org/grpc/metadata" +) + +type Client struct { + config *Config + log logrus.FieldLogger + + conn *grpc.ClientConn + pb xatu.CoordinatorClient +} + +func New(config *Config, log logrus.FieldLogger) (*Client, error) { + if config == nil { + return nil, errors.New("config is required") + } + + if err := config.Validate(); err != nil { + return nil, err + } + + var opts []grpc.DialOption + + if config.TLS { + host, _, err := net.SplitHostPort(config.Address) + if err != nil { + return nil, fmt.Errorf("fail to get host from address: %v", err) + } + + opts = append(opts, grpc.WithTransportCredentials(credentials.NewClientTLSFromCert(nil, host))) + } else { + opts = append(opts, grpc.WithTransportCredentials(insecure.NewCredentials())) + } + + conn, err := grpc.Dial(config.Address, opts...) + if err != nil { + return nil, fmt.Errorf("fail to dial: %v", err) + } + + pbClient := xatu.NewCoordinatorClient(conn) + + return &Client{ + config: config, + log: log, + conn: conn, + pb: pbClient, + }, nil +} + +func (c *Client) Start(ctx context.Context) error { + return nil +} + +func (c *Client) Stop(ctx context.Context) error { + if err := c.conn.Close(); err != nil { + return err + } + + return nil +} + +func (c *Client) GetCannonLocation(ctx context.Context, typ xatu.CannonType, networkID string) (*xatu.CannonLocation, error) { + req := xatu.GetCannonLocationRequest{ + Type: typ, + NetworkId: networkID, + } + + md := metadata.New(c.config.Headers) + ctx = metadata.NewOutgoingContext(ctx, md) + + res, err := c.pb.GetCannonLocation(ctx, &req, grpc.UseCompressor(gzip.Name)) + if err != nil { + return nil, err + } + + return res.Location, nil +} + +func (c *Client) UpsertCannonLocationRequest(ctx context.Context, location *xatu.CannonLocation) error { + req := xatu.UpsertCannonLocationRequest{ + Location: location, + } + + md := metadata.New(c.config.Headers) + ctx = metadata.NewOutgoingContext(ctx, md) + + _, err := c.pb.UpsertCannonLocation(ctx, &req, grpc.UseCompressor(gzip.Name)) + if err != nil { + return err + } + + return nil +} diff --git a/pkg/cannon/coordinator/config.go b/pkg/cannon/coordinator/config.go new file mode 100644 index 00000000..98f32409 --- /dev/null +++ b/pkg/cannon/coordinator/config.go @@ -0,0 +1,19 @@ +package coordinator + +import ( + "errors" +) + +type Config struct { + Address string `yaml:"address"` + Headers map[string]string `yaml:"headers"` + TLS bool `yaml:"tls" default:"false"` +} + +func (c *Config) Validate() error { + if c.Address == "" { + return errors.New("address is required") + } + + return nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/attester_slashing.go b/pkg/cannon/deriver/beacon/eth/v2/attester_slashing.go new file mode 100644 index 00000000..d8d85d25 --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/attester_slashing.go @@ -0,0 +1,250 @@ +package v2 + +import ( + "context" + "time" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/attestantio/go-eth2-client/spec/phase0" + backoff "github.com/cenkalti/backoff/v4" + "github.com/ethpandaops/xatu/pkg/cannon/ethereum" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" + xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/google/uuid" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + AttesterSlashingDeriverName = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING +) + +type AttesterSlashingDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` + HeadSlotLag *uint64 `yaml:"headSlotLag" default:"1"` +} + +type AttesterSlashingDeriver struct { + log logrus.FieldLogger + cfg *AttesterSlashingDeriverConfig + iterator *iterator.SlotIterator + onEventCallbacks []func(ctx context.Context, event *xatu.DecoratedEvent) error + onLocationCallbacks []func(ctx context.Context, loc uint64) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +func NewAttesterSlashingDeriver(log logrus.FieldLogger, config *AttesterSlashingDeriverConfig, iter *iterator.SlotIterator, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *AttesterSlashingDeriver { + return &AttesterSlashingDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v2/attester_slashing"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (a *AttesterSlashingDeriver) CannonType() xatu.CannonType { + return AttesterSlashingDeriverName +} + +func (a *AttesterSlashingDeriver) Name() string { + return AttesterSlashingDeriverName.String() +} + +func (a *AttesterSlashingDeriver) OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) { + a.onEventCallbacks = append(a.onEventCallbacks, fn) +} + +func (a *AttesterSlashingDeriver) OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, loc uint64) error) { + a.onLocationCallbacks = append(a.onLocationCallbacks, fn) +} + +func (a *AttesterSlashingDeriver) Start(ctx context.Context) error { + if !a.cfg.Enabled { + a.log.Info("Attester slashing deriver disabled") + + return nil + } + + a.log.Info("Attester slashing deriver enabled") + + // Start our main loop + go a.run(ctx) + + return nil +} + +func (a *AttesterSlashingDeriver) Stop(ctx context.Context) error { + return nil +} + +func (a *AttesterSlashingDeriver) run(ctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 1 * time.Minute + + for { + select { + case <-ctx.Done(): + return + default: + operation := func() error { + time.Sleep(100 * time.Millisecond) + + // Get the next slot + location, err := a.iterator.Next(ctx) + if err != nil { + return err + } + + for _, fn := range a.onLocationCallbacks { + if errr := fn(ctx, location.GetEthV2BeaconBlockAttesterSlashing().GetSlot()); errr != nil { + a.log.WithError(errr).Error("Failed to send location") + } + } + + // Process the slot + events, err := a.processSlot(ctx, phase0.Slot(location.GetEthV2BeaconBlockAttesterSlashing().GetSlot())) + if err != nil { + a.log.WithError(err).Error("Failed to process slot") + + return err + } + + // Send the events + for _, event := range events { + for _, fn := range a.onEventCallbacks { + if err := fn(ctx, event); err != nil { + a.log.WithError(err).Error("Failed to send event") + } + } + } + + // Update our location + if err := a.iterator.UpdateLocation(ctx, location); err != nil { + return err + } + + bo.Reset() + + return nil + } + + if err := backoff.RetryNotify(operation, bo, func(err error, timer time.Duration) { + a.log.WithError(err).WithField("next_attempt", timer).Warn("Failed to process") + }); err != nil { + a.log.WithError(err).Warn("Failed to process") + } + } + } +} + +func (a *AttesterSlashingDeriver) processSlot(ctx context.Context, slot phase0.Slot) ([]*xatu.DecoratedEvent, error) { + // Get the block + block, err := a.beacon.GetBeaconBlock(ctx, xatuethv1.SlotAsString(slot)) + if err != nil { + return nil, errors.Wrapf(err, "failed to get beacon block for slot %d", slot) + } + + if block == nil { + return []*xatu.DecoratedEvent{}, nil + } + + blockIdentifier, err := GetBlockIdentifier(block, a.beacon.Metadata().Wallclock()) + if err != nil { + return nil, errors.Wrapf(err, "failed to get block identifier for slot %d", slot) + } + + events := []*xatu.DecoratedEvent{} + + for _, slashing := range a.getAttesterSlashings(ctx, block) { + event, err := a.createEvent(ctx, slashing, blockIdentifier) + if err != nil { + a.log.WithError(err).Error("Failed to create event") + + return nil, errors.Wrapf(err, "failed to create event for attester slashing %s", slashing.String()) + } + + events = append(events, event) + } + + return events, nil +} + +func (a *AttesterSlashingDeriver) getAttesterSlashings(ctx context.Context, block *spec.VersionedSignedBeaconBlock) []*xatuethv1.AttesterSlashingV2 { + slashings := []*xatuethv1.AttesterSlashingV2{} + + attesterSlashings, err := block.AttesterSlashings() + if err != nil { + a.log.WithError(err).Error("Failed to obtain attester slashings") + } + + for _, slashing := range attesterSlashings { + slashings = append(slashings, &xatuethv1.AttesterSlashingV2{ + Attestation_1: convertIndexedAttestation(slashing.Attestation1), + Attestation_2: convertIndexedAttestation(slashing.Attestation2), + }) + } + + return slashings +} + +func convertIndexedAttestation(attestation *phase0.IndexedAttestation) *xatuethv1.IndexedAttestationV2 { + indicies := []*wrapperspb.UInt64Value{} + + for _, index := range attestation.AttestingIndices { + indicies = append(indicies, &wrapperspb.UInt64Value{Value: index}) + } + + return &xatuethv1.IndexedAttestationV2{ + AttestingIndices: indicies, + Data: &xatuethv1.AttestationDataV2{ + Slot: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Slot)}, + Index: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Index)}, + BeaconBlockRoot: attestation.Data.BeaconBlockRoot.String(), + Source: &xatuethv1.CheckpointV2{ + Epoch: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Source.Epoch)}, + Root: attestation.Data.Source.Root.String(), + }, + Target: &xatuethv1.CheckpointV2{ + Epoch: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Target.Epoch)}, + Root: attestation.Data.Target.Root.String(), + }, + }, + Signature: attestation.Signature.String(), + } +} + +func (a *AttesterSlashingDeriver) createEvent(ctx context.Context, slashing *xatuethv1.AttesterSlashingV2, identifier *xatu.BlockIdentifier) (*xatu.DecoratedEvent, error) { + // Make a clone of the metadata + metadata, ok := proto.Clone(a.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV2BeaconBlockAttesterSlashing{ + EthV2BeaconBlockAttesterSlashing: slashing, + }, + } + + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockAttesterSlashing{ + EthV2BeaconBlockAttesterSlashing: &xatu.ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData{ + Block: identifier, + }, + } + + return decoratedEvent, nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/beacon_block.go b/pkg/cannon/deriver/beacon/eth/v2/beacon_block.go new file mode 100644 index 00000000..4783224b --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/beacon_block.go @@ -0,0 +1,44 @@ +package v2 + +import ( + "fmt" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/ethpandaops/ethwallclock" + v1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +func GetBlockIdentifier(block *spec.VersionedSignedBeaconBlock, wallclock *ethwallclock.EthereumBeaconChain) (*xatu.BlockIdentifier, error) { + if block == nil { + return nil, fmt.Errorf("block is nil") + } + + slotNum, err := block.Slot() + if err != nil { + return nil, err + } + + root, err := block.Root() + if err != nil { + return nil, err + } + + slot := wallclock.Slots().FromNumber(uint64(slotNum)) + epoch := wallclock.Epochs().FromSlot(uint64(slotNum)) + + return &xatu.BlockIdentifier{ + Epoch: &xatu.EpochV2{ + Number: &wrapperspb.UInt64Value{Value: epoch.Number()}, + StartDateTime: timestamppb.New(epoch.TimeWindow().Start()), + }, + Slot: &xatu.SlotV2{ + Number: &wrapperspb.UInt64Value{Value: slot.Number()}, + StartDateTime: timestamppb.New(slot.TimeWindow().Start()), + }, + Root: v1.RootAsString(root), + Version: block.Version.String(), + }, nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/bls_to_execution_change.go b/pkg/cannon/deriver/beacon/eth/v2/bls_to_execution_change.go new file mode 100644 index 00000000..e2d6147c --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/bls_to_execution_change.go @@ -0,0 +1,242 @@ +package v2 + +import ( + "context" + "fmt" + "time" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/attestantio/go-eth2-client/spec/phase0" + backoff "github.com/cenkalti/backoff/v4" + "github.com/ethpandaops/xatu/pkg/cannon/ethereum" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" + xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + xatuethv2 "github.com/ethpandaops/xatu/pkg/proto/eth/v2" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/google/uuid" + "github.com/pkg/errors" + + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + BLSToExecutionChangeDeriverName = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE +) + +type BLSToExecutionChangeDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` + HeadSlotLag *uint64 `yaml:"headSlotLag" default:"1"` +} + +type BLSToExecutionChangeDeriver struct { + log logrus.FieldLogger + cfg *BLSToExecutionChangeDeriverConfig + iterator *iterator.SlotIterator + onEventCallbacks []func(ctx context.Context, event *xatu.DecoratedEvent) error + onLocationCallbacks []func(ctx context.Context, loc uint64) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +func NewBLSToExecutionChangeDeriver(log logrus.FieldLogger, config *BLSToExecutionChangeDeriverConfig, iter *iterator.SlotIterator, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *BLSToExecutionChangeDeriver { + return &BLSToExecutionChangeDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v2/bls_to_execution_change"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (b *BLSToExecutionChangeDeriver) CannonType() xatu.CannonType { + return BLSToExecutionChangeDeriverName +} + +func (b *BLSToExecutionChangeDeriver) Name() string { + return BLSToExecutionChangeDeriverName.String() +} + +func (b *BLSToExecutionChangeDeriver) OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) { + b.onEventCallbacks = append(b.onEventCallbacks, fn) +} + +func (b *BLSToExecutionChangeDeriver) OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, location uint64) error) { + b.onLocationCallbacks = append(b.onLocationCallbacks, fn) +} + +func (b *BLSToExecutionChangeDeriver) Start(ctx context.Context) error { + if !b.cfg.Enabled { + b.log.Info("BLS to execution change deriver disabled") + + return nil + } + + b.log.Info("BLS to execution change deriver enabled") + + // Start our main loop + go b.run(ctx) + + return nil +} + +func (b *BLSToExecutionChangeDeriver) Stop(ctx context.Context) error { + return nil +} + +func (b *BLSToExecutionChangeDeriver) run(ctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 1 * time.Minute + + for { + select { + case <-ctx.Done(): + return + default: + operation := func() error { + time.Sleep(100 * time.Millisecond) + + // Get the next slot + location, err := b.iterator.Next(ctx) + if err != nil { + return err + } + + for _, fn := range b.onLocationCallbacks { + if errr := fn(ctx, location.GetEthV2BeaconBlockBlsToExecutionChange().GetSlot()); errr != nil { + b.log.WithError(errr).Error("Failed to send location") + } + } + + // Process the slot + events, err := b.processSlot(ctx, phase0.Slot(location.GetEthV2BeaconBlockBlsToExecutionChange().GetSlot())) + if err != nil { + b.log.WithError(err).Error("Failed to process slot") + + return err + } + + // Send the events + for _, event := range events { + for _, fn := range b.onEventCallbacks { + if err := fn(ctx, event); err != nil { + b.log.WithError(err).Error("Failed to send event") + } + } + } + + // Update our location + if err := b.iterator.UpdateLocation(ctx, location); err != nil { + return err + } + + bo.Reset() + + return nil + } + + if err := backoff.RetryNotify(operation, bo, func(err error, timer time.Duration) { + b.log.WithError(err).WithField("next_attempt", timer).Warn("Failed to process") + }); err != nil { + b.log.WithError(err).Warn("Failed to process") + } + } + } +} + +func (b *BLSToExecutionChangeDeriver) processSlot(ctx context.Context, slot phase0.Slot) ([]*xatu.DecoratedEvent, error) { + // Get the block + block, err := b.beacon.GetBeaconBlock(ctx, xatuethv1.SlotAsString(slot)) + if err != nil { + return nil, errors.Wrapf(err, "failed to get beacon block for slot %d", slot) + } + + if block == nil { + return []*xatu.DecoratedEvent{}, nil + } + + blockIdentifier, err := GetBlockIdentifier(block, b.beacon.Metadata().Wallclock()) + if err != nil { + return nil, errors.Wrapf(err, "failed to get block identifier for slot %d", slot) + } + + events := []*xatu.DecoratedEvent{} + + changes, err := b.getBLSToExecutionChanges(ctx, block) + if err != nil { + return nil, err + } + + for _, change := range changes { + event, err := b.createEvent(ctx, change, blockIdentifier) + if err != nil { + b.log.WithError(err).Error("Failed to create event") + + return nil, errors.Wrapf(err, "failed to create event for BLS to execution change %s", change.String()) + } + + events = append(events, event) + } + + return events, nil +} + +func (b *BLSToExecutionChangeDeriver) getBLSToExecutionChanges(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv2.SignedBLSToExecutionChangeV2, error) { + changes := []*xatuethv2.SignedBLSToExecutionChangeV2{} + + switch block.Version { + case spec.DataVersionPhase0: + return changes, nil + case spec.DataVersionAltair: + return changes, nil + case spec.DataVersionBellatrix: + return changes, nil + case spec.DataVersionCapella: + for _, change := range block.Capella.Message.Body.BLSToExecutionChanges { + changes = append(changes, &xatuethv2.SignedBLSToExecutionChangeV2{ + Message: &xatuethv2.BLSToExecutionChangeV2{ + ValidatorIndex: wrapperspb.UInt64(uint64(change.Message.ValidatorIndex)), + FromBlsPubkey: change.Message.FromBLSPubkey.String(), + ToExecutionAddress: change.Message.ToExecutionAddress.String(), + }, + Signature: change.Signature.String(), + }) + } + default: + return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) + } + + return changes, nil +} + +func (b *BLSToExecutionChangeDeriver) createEvent(ctx context.Context, change *xatuethv2.SignedBLSToExecutionChangeV2, identifier *xatu.BlockIdentifier) (*xatu.DecoratedEvent, error) { + // Make a clone of the metadata + metadata, ok := proto.Clone(b.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange{ + EthV2BeaconBlockBlsToExecutionChange: change, + }, + } + + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockBlsToExecutionChange{ + EthV2BeaconBlockBlsToExecutionChange: &xatu.ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData{ + Block: identifier, + }, + } + + return decoratedEvent, nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/deposit.go b/pkg/cannon/deriver/beacon/eth/v2/deposit.go new file mode 100644 index 00000000..cc05f200 --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/deposit.go @@ -0,0 +1,250 @@ +package v2 + +import ( + "context" + "fmt" + "time" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/attestantio/go-eth2-client/spec/phase0" + backoff "github.com/cenkalti/backoff/v4" + "github.com/ethpandaops/xatu/pkg/cannon/ethereum" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" + xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/google/uuid" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + DepositDeriverName = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT +) + +type DepositDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` + HeadSlotLag *uint64 `yaml:"headSlotLag" default:"1"` +} + +type DepositDeriver struct { + log logrus.FieldLogger + cfg *DepositDeriverConfig + iterator *iterator.SlotIterator + onEventCallbacks []func(ctx context.Context, event *xatu.DecoratedEvent) error + onLocationCallbacks []func(ctx context.Context, location uint64) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +func NewDepositDeriver(log logrus.FieldLogger, config *DepositDeriverConfig, iter *iterator.SlotIterator, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *DepositDeriver { + return &DepositDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v2/deposit"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (b *DepositDeriver) CannonType() xatu.CannonType { + return DepositDeriverName +} + +func (b *DepositDeriver) Name() string { + return DepositDeriverName.String() +} + +func (b *DepositDeriver) OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) { + b.onEventCallbacks = append(b.onEventCallbacks, fn) +} + +func (b *DepositDeriver) OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, location uint64) error) { + b.onLocationCallbacks = append(b.onLocationCallbacks, fn) +} + +func (b *DepositDeriver) Start(ctx context.Context) error { + if !b.cfg.Enabled { + b.log.Info("Deposit deriver disabled") + + return nil + } + + b.log.Info("Deposit deriver enabled") + + // Start our main loop + go b.run(ctx) + + return nil +} + +func (b *DepositDeriver) Stop(ctx context.Context) error { + return nil +} + +func (b *DepositDeriver) run(ctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 1 * time.Minute + + for { + select { + case <-ctx.Done(): + return + default: + operation := func() error { + time.Sleep(100 * time.Millisecond) + + // Get the next slot + location, err := b.iterator.Next(ctx) + if err != nil { + return err + } + + for _, fn := range b.onLocationCallbacks { + if errr := fn(ctx, location.GetEthV2BeaconBlockDeposit().GetSlot()); errr != nil { + b.log.WithError(errr).Error("Failed to send location") + } + } + + // Process the slot + events, err := b.processSlot(ctx, phase0.Slot(location.GetEthV2BeaconBlockDeposit().GetSlot())) + if err != nil { + b.log.WithError(err).Error("Failed to process slot") + + return err + } + + // Send the events + for _, event := range events { + for _, fn := range b.onEventCallbacks { + if err := fn(ctx, event); err != nil { + b.log.WithError(err).Error("Failed to send event") + } + } + } + + // Update our location + if err := b.iterator.UpdateLocation(ctx, location); err != nil { + return err + } + + bo.Reset() + + return nil + } + + if err := backoff.RetryNotify(operation, bo, func(err error, timer time.Duration) { + b.log.WithError(err).WithField("next_attempt", timer).Warn("Failed to process") + }); err != nil { + b.log.WithError(err).Warn("Failed to process") + } + } + } +} + +func (b *DepositDeriver) processSlot(ctx context.Context, slot phase0.Slot) ([]*xatu.DecoratedEvent, error) { + // Get the block + block, err := b.beacon.GetBeaconBlock(ctx, xatuethv1.SlotAsString(slot)) + if err != nil { + return nil, errors.Wrapf(err, "failed to get beacon block for slot %d", slot) + } + + if block == nil { + return []*xatu.DecoratedEvent{}, nil + } + + blockIdentifier, err := GetBlockIdentifier(block, b.beacon.Metadata().Wallclock()) + if err != nil { + return nil, errors.Wrapf(err, "failed to get block identifier for slot %d", slot) + } + + events := []*xatu.DecoratedEvent{} + + deposits, err := b.getDeposits(ctx, block) + if err != nil { + return nil, errors.Wrapf(err, "failed to get deposits for block %s", blockIdentifier.String()) + } + + for _, deposit := range deposits { + event, err := b.createEvent(ctx, deposit, blockIdentifier) + if err != nil { + b.log.WithError(err).Error("Failed to create event") + + return nil, errors.Wrapf(err, "failed to create event for deposit %s", deposit.String()) + } + + events = append(events, event) + } + + return events, nil +} + +func (b *DepositDeriver) getDeposits(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv1.DepositV2, error) { + exits := []*xatuethv1.DepositV2{} + + var deposits []*phase0.Deposit + + switch block.Version { + case spec.DataVersionPhase0: + deposits = block.Phase0.Message.Body.Deposits + case spec.DataVersionAltair: + deposits = block.Altair.Message.Body.Deposits + case spec.DataVersionBellatrix: + deposits = block.Bellatrix.Message.Body.Deposits + case spec.DataVersionCapella: + deposits = block.Capella.Message.Body.Deposits + default: + return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) + } + + for _, deposit := range deposits { + proof := []string{} + for _, p := range deposit.Proof { + proof = append(proof, fmt.Sprintf("0x%x", p)) + } + + exits = append(exits, &xatuethv1.DepositV2{ + Proof: proof, + Data: &xatuethv1.DepositV2_Data{ + Pubkey: deposit.Data.PublicKey.String(), + WithdrawalCredentials: fmt.Sprintf("0x%x", deposit.Data.WithdrawalCredentials), + Amount: wrapperspb.UInt64(uint64(deposit.Data.Amount)), + Signature: deposit.Data.Signature.String(), + }, + }) + } + + return exits, nil +} + +func (b *DepositDeriver) createEvent(ctx context.Context, deposit *xatuethv1.DepositV2, identifier *xatu.BlockIdentifier) (*xatu.DecoratedEvent, error) { + // Make a clone of the metadata + metadata, ok := proto.Clone(b.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV2BeaconBlockDeposit{ + EthV2BeaconBlockDeposit: deposit, + }, + } + + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockDeposit{ + EthV2BeaconBlockDeposit: &xatu.ClientMeta_AdditionalEthV2BeaconBlockDepositData{ + Block: identifier, + }, + } + + return decoratedEvent, nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/execution_transaction.go b/pkg/cannon/deriver/beacon/eth/v2/execution_transaction.go new file mode 100644 index 00000000..53c31ab7 --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/execution_transaction.go @@ -0,0 +1,290 @@ +package v2 + +import ( + "context" + "encoding/hex" + "fmt" + "strconv" + "time" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/attestantio/go-eth2-client/spec/phase0" + backoff "github.com/cenkalti/backoff/v4" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethpandaops/xatu/pkg/cannon/ethereum" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" + xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/google/uuid" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +type ExecutionTransactionDeriver struct { + log logrus.FieldLogger + cfg *ExecutionTransactionDeriverConfig + iterator *iterator.SlotIterator + onEventCallbacks []func(ctx context.Context, event *xatu.DecoratedEvent) error + onLocationCallbacks []func(ctx context.Context, location uint64) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +type ExecutionTransactionDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` + HeadSlotLag *uint64 `yaml:"headSlotLag" default:"1"` +} + +const ( + ExecutionTransactionDeriverName = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION +) + +func NewExecutionTransactionDeriver(log logrus.FieldLogger, config *ExecutionTransactionDeriverConfig, iter *iterator.SlotIterator, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *ExecutionTransactionDeriver { + return &ExecutionTransactionDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v2/execution_transaction"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (b *ExecutionTransactionDeriver) CannonType() xatu.CannonType { + return ExecutionTransactionDeriverName +} + +func (b *ExecutionTransactionDeriver) Name() string { + return ExecutionTransactionDeriverName.String() +} + +func (b *ExecutionTransactionDeriver) OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) { + b.onEventCallbacks = append(b.onEventCallbacks, fn) +} + +func (b *ExecutionTransactionDeriver) OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, location uint64) error) { + b.onLocationCallbacks = append(b.onLocationCallbacks, fn) +} + +func (b *ExecutionTransactionDeriver) Start(ctx context.Context) error { + if !b.cfg.Enabled { + b.log.Info("Execution transaction deriver disabled") + + return nil + } + + b.log.Info("Execution transaction deriver enabled") + + // Start our main loop + go b.run(ctx) + + return nil +} + +func (b *ExecutionTransactionDeriver) Stop(ctx context.Context) error { + return nil +} + +func (b *ExecutionTransactionDeriver) run(ctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 1 * time.Minute + + for { + select { + case <-ctx.Done(): + return + default: + operation := func() error { + time.Sleep(100 * time.Millisecond) + + // Get the next slot + location, err := b.iterator.Next(ctx) + if err != nil { + return err + } + + for _, fn := range b.onLocationCallbacks { + if errr := fn(ctx, location.GetEthV2BeaconBlockExecutionTransaction().GetSlot()); errr != nil { + b.log.WithError(errr).Error("Failed to send location") + } + } + + // Process the slot + events, err := b.processSlot(ctx, phase0.Slot(location.GetEthV2BeaconBlockExecutionTransaction().GetSlot())) + if err != nil { + b.log.WithError(err).Error("Failed to process slot") + + return err + } + + // Send the events + for _, event := range events { + for _, fn := range b.onEventCallbacks { + if err := fn(ctx, event); err != nil { + b.log.WithError(err).Error("Failed to send event") + } + } + } + + // Update our location + if err := b.iterator.UpdateLocation(ctx, location); err != nil { + return err + } + + bo.Reset() + + return nil + } + + if err := backoff.RetryNotify(operation, bo, func(err error, timer time.Duration) { + b.log.WithError(err).WithField("next_attempt", timer).Warn("Failed to process") + }); err != nil { + b.log.WithError(err).Warn("Failed to process") + } + } + } +} + +func (b *ExecutionTransactionDeriver) processSlot(ctx context.Context, slot phase0.Slot) ([]*xatu.DecoratedEvent, error) { + // Get the block + block, err := b.beacon.GetBeaconBlock(ctx, xatuethv1.SlotAsString(slot)) + if err != nil { + return nil, errors.Wrapf(err, "failed to get beacon block for slot %d", slot) + } + + if block == nil { + return []*xatu.DecoratedEvent{}, nil + } + + blockIdentifier, err := GetBlockIdentifier(block, b.beacon.Metadata().Wallclock()) + if err != nil { + return nil, errors.Wrapf(err, "failed to get block identifier for slot %d", slot) + } + + events := []*xatu.DecoratedEvent{} + + transactions, err := b.getExecutionTransactions(ctx, block) + if err != nil { + return nil, err + } + + for index, transaction := range transactions { + from, err := types.Sender(types.LatestSignerForChainID(transaction.ChainId()), transaction) + if err != nil { + return nil, fmt.Errorf("failed to get transaction sender: %v", err) + } + + gasPrice := transaction.GasPrice() + if gasPrice == nil { + return nil, fmt.Errorf("failed to get transaction gas price") + } + + value := transaction.Value() + if value == nil { + return nil, fmt.Errorf("failed to get transaction value") + } + + to := "" + + if transaction.To() != nil { + to = transaction.To().Hex() + } + + chainID := transaction.ChainId() + if chainID == nil { + return nil, fmt.Errorf("failed to get transaction chain ID") + } + + tx := &xatuethv1.Transaction{ + Nonce: wrapperspb.UInt64(transaction.Nonce()), + GasPrice: gasPrice.String(), + Gas: wrapperspb.UInt64(transaction.Gas()), + To: to, + From: from.Hex(), + Value: value.String(), + Input: hex.EncodeToString(transaction.Data()), + Hash: transaction.Hash().Hex(), + ChainId: chainID.String(), + Type: wrapperspb.UInt32(uint32(transaction.Type())), + } + + event, err := b.createEvent(ctx, tx, uint64(index), blockIdentifier, transaction) + if err != nil { + b.log.WithError(err).Error("Failed to create event") + + return nil, errors.Wrapf(err, "failed to create event for execution transaction %s", transaction.Hash()) + } + + events = append(events, event) + } + + return events, nil +} + +func (b *ExecutionTransactionDeriver) getExecutionTransactions(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*types.Transaction, error) { + transactions := []*types.Transaction{} + + switch block.Version { + case spec.DataVersionPhase0: + return transactions, nil + case spec.DataVersionAltair: + return transactions, nil + case spec.DataVersionBellatrix: + for _, transaction := range block.Bellatrix.Message.Body.ExecutionPayload.Transactions { + ethTransaction := new(types.Transaction) + if err := ethTransaction.UnmarshalBinary(transaction); err != nil { + return nil, fmt.Errorf("failed to unmarshal transaction: %v", err) + } + + transactions = append(transactions, ethTransaction) + } + case spec.DataVersionCapella: + for _, transaction := range block.Capella.Message.Body.ExecutionPayload.Transactions { + ethTransaction := new(types.Transaction) + if err := ethTransaction.UnmarshalBinary(transaction); err != nil { + return nil, fmt.Errorf("failed to unmarshal transaction: %v", err) + } + + transactions = append(transactions, ethTransaction) + } + default: + return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) + } + + return transactions, nil +} + +func (b *ExecutionTransactionDeriver) createEvent(ctx context.Context, transaction *xatuethv1.Transaction, positionInBlock uint64, blockIdentifier *xatu.BlockIdentifier, rlpTransaction *types.Transaction) (*xatu.DecoratedEvent, error) { + // Make a clone of the metadata + metadata, ok := proto.Clone(b.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV2BeaconBlockExecutionTransaction{ + EthV2BeaconBlockExecutionTransaction: transaction, + }, + } + + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockExecutionTransaction{ + EthV2BeaconBlockExecutionTransaction: &xatu.ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData{ + Block: blockIdentifier, + PositionInBlock: wrapperspb.UInt64(positionInBlock), + Size: strconv.FormatFloat(float64(rlpTransaction.Size()), 'f', 0, 64), + CallDataSize: fmt.Sprintf("%d", len(rlpTransaction.Data())), + }, + } + + return decoratedEvent, nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/proposer_slashing.go b/pkg/cannon/deriver/beacon/eth/v2/proposer_slashing.go new file mode 100644 index 00000000..bac6ff87 --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/proposer_slashing.go @@ -0,0 +1,247 @@ +package v2 + +import ( + "context" + "time" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/attestantio/go-eth2-client/spec/phase0" + backoff "github.com/cenkalti/backoff/v4" + "github.com/ethpandaops/xatu/pkg/cannon/ethereum" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" + xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/google/uuid" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + ProposerSlashingDeriverName = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING +) + +type ProposerSlashingDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` + HeadSlotLag *uint64 `yaml:"headSlotLag" default:"1"` +} + +type ProposerSlashingDeriver struct { + log logrus.FieldLogger + cfg *ProposerSlashingDeriverConfig + iterator *iterator.SlotIterator + onEventCallbacks []func(ctx context.Context, event *xatu.DecoratedEvent) error + onLocationCallbacks []func(ctx context.Context, location uint64) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +func NewProposerSlashingDeriver(log logrus.FieldLogger, config *ProposerSlashingDeriverConfig, iter *iterator.SlotIterator, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *ProposerSlashingDeriver { + return &ProposerSlashingDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v2/proposer_slashing"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (b *ProposerSlashingDeriver) CannonType() xatu.CannonType { + return ProposerSlashingDeriverName +} + +func (b *ProposerSlashingDeriver) Name() string { + return ProposerSlashingDeriverName.String() +} + +func (b *ProposerSlashingDeriver) OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) { + b.onEventCallbacks = append(b.onEventCallbacks, fn) +} + +func (b *ProposerSlashingDeriver) OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, location uint64) error) { + b.onLocationCallbacks = append(b.onLocationCallbacks, fn) +} + +func (b *ProposerSlashingDeriver) Start(ctx context.Context) error { + if !b.cfg.Enabled { + b.log.Info("Proposer slashing deriver disabled") + + return nil + } + + b.log.Info("Proposer slashing deriver enabled") + + // Start our main loop + go b.run(ctx) + + return nil +} + +func (b *ProposerSlashingDeriver) Stop(ctx context.Context) error { + return nil +} + +func (b *ProposerSlashingDeriver) run(ctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 1 * time.Minute + + for { + select { + case <-ctx.Done(): + return + default: + operation := func() error { + time.Sleep(100 * time.Millisecond) + + // Get the next slot + location, err := b.iterator.Next(ctx) + if err != nil { + return err + } + + for _, fn := range b.onLocationCallbacks { + if errr := fn(ctx, location.GetEthV2BeaconBlockProposerSlashing().GetSlot()); errr != nil { + b.log.WithError(errr).Error("Failed to send location") + } + } + + // Process the slot + events, err := b.processSlot(ctx, phase0.Slot(location.GetEthV2BeaconBlockProposerSlashing().GetSlot())) + if err != nil { + b.log.WithError(err).Error("Failed to process slot") + + return err + } + + // Send the events + for _, event := range events { + for _, fn := range b.onEventCallbacks { + if err := fn(ctx, event); err != nil { + b.log.WithError(err).Error("Failed to send event") + } + } + } + + // Update our location + if err := b.iterator.UpdateLocation(ctx, location); err != nil { + return err + } + + bo.Reset() + + return nil + } + + if err := backoff.RetryNotify(operation, bo, func(err error, timer time.Duration) { + b.log.WithError(err).WithField("next_attempt", timer).Warn("Failed to process") + }); err != nil { + b.log.WithError(err).Warn("Failed to process") + } + } + } +} + +func (b *ProposerSlashingDeriver) processSlot(ctx context.Context, slot phase0.Slot) ([]*xatu.DecoratedEvent, error) { + // Get the block + block, err := b.beacon.GetBeaconBlock(ctx, xatuethv1.SlotAsString(slot)) + if err != nil { + return nil, errors.Wrapf(err, "failed to get beacon block for slot %d", slot) + } + + if block == nil { + return []*xatu.DecoratedEvent{}, nil + } + + blockIdentifier, err := GetBlockIdentifier(block, b.beacon.Metadata().Wallclock()) + if err != nil { + return nil, errors.Wrapf(err, "failed to get block identifier for slot %d", slot) + } + + events := []*xatu.DecoratedEvent{} + + slashings, err := b.getProposerSlashings(ctx, block) + if err != nil { + return nil, err + } + + for _, slashing := range slashings { + event, err := b.createEvent(ctx, slashing, blockIdentifier) + if err != nil { + b.log.WithError(err).Error("Failed to create event") + + return nil, errors.Wrapf(err, "failed to create event for proposer slashing %s", slashing.String()) + } + + events = append(events, event) + } + + return events, nil +} + +func (b *ProposerSlashingDeriver) getProposerSlashings(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv1.ProposerSlashingV2, error) { + slashings := []*xatuethv1.ProposerSlashingV2{} + + blockSlashings, err := block.ProposerSlashings() + if err != nil { + return nil, err + } + + for _, slashing := range blockSlashings { + slashings = append(slashings, &xatuethv1.ProposerSlashingV2{ + SignedHeader_1: &xatuethv1.SignedBeaconBlockHeaderV2{ + Message: &xatuethv1.BeaconBlockHeaderV2{ + Slot: wrapperspb.UInt64(uint64(slashing.SignedHeader1.Message.Slot)), + ProposerIndex: wrapperspb.UInt64(uint64(slashing.SignedHeader1.Message.ProposerIndex)), + ParentRoot: slashing.SignedHeader1.Message.ParentRoot.String(), + StateRoot: slashing.SignedHeader1.Message.StateRoot.String(), + BodyRoot: slashing.SignedHeader1.Message.BodyRoot.String(), + }, + Signature: slashing.SignedHeader1.Signature.String(), + }, + SignedHeader_2: &xatuethv1.SignedBeaconBlockHeaderV2{ + Message: &xatuethv1.BeaconBlockHeaderV2{ + Slot: wrapperspb.UInt64(uint64(slashing.SignedHeader2.Message.Slot)), + ProposerIndex: wrapperspb.UInt64(uint64(slashing.SignedHeader2.Message.ProposerIndex)), + ParentRoot: slashing.SignedHeader2.Message.ParentRoot.String(), + StateRoot: slashing.SignedHeader2.Message.StateRoot.String(), + BodyRoot: slashing.SignedHeader2.Message.BodyRoot.String(), + }, + Signature: slashing.SignedHeader2.Signature.String(), + }, + }) + } + + return slashings, nil +} + +func (b *ProposerSlashingDeriver) createEvent(ctx context.Context, slashing *xatuethv1.ProposerSlashingV2, identifier *xatu.BlockIdentifier) (*xatu.DecoratedEvent, error) { + // Make a clone of the metadata + metadata, ok := proto.Clone(b.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV2BeaconBlockProposerSlashing{ + EthV2BeaconBlockProposerSlashing: slashing, + }, + } + + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockProposerSlashing{ + EthV2BeaconBlockProposerSlashing: &xatu.ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData{ + Block: identifier, + }, + } + + return decoratedEvent, nil +} diff --git a/pkg/cannon/deriver/beacon/eth/v2/voluntary_exit.go b/pkg/cannon/deriver/beacon/eth/v2/voluntary_exit.go new file mode 100644 index 00000000..6bd50cd8 --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v2/voluntary_exit.go @@ -0,0 +1,243 @@ +package v2 + +import ( + "context" + "fmt" + "time" + + "github.com/attestantio/go-eth2-client/spec" + "github.com/attestantio/go-eth2-client/spec/phase0" + backoff "github.com/cenkalti/backoff/v4" + "github.com/ethpandaops/xatu/pkg/cannon/ethereum" + "github.com/ethpandaops/xatu/pkg/cannon/iterator" + xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/google/uuid" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + VoluntaryExitDeriverName = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT +) + +type VoluntaryExitDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` + HeadSlotLag *uint64 `yaml:"headSlotLag" default:"1"` +} + +type VoluntaryExitDeriver struct { + log logrus.FieldLogger + cfg *VoluntaryExitDeriverConfig + iterator *iterator.SlotIterator + onEventCallbacks []func(ctx context.Context, event *xatu.DecoratedEvent) error + onLocationCallbacks []func(ctx context.Context, location uint64) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +func NewVoluntaryExitDeriver(log logrus.FieldLogger, config *VoluntaryExitDeriverConfig, iter *iterator.SlotIterator, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *VoluntaryExitDeriver { + return &VoluntaryExitDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v2/voluntary_exit"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (b *VoluntaryExitDeriver) CannonType() xatu.CannonType { + return VoluntaryExitDeriverName +} + +func (b *VoluntaryExitDeriver) Name() string { + return VoluntaryExitDeriverName.String() +} + +func (b *VoluntaryExitDeriver) OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) { + b.onEventCallbacks = append(b.onEventCallbacks, fn) +} + +func (b *VoluntaryExitDeriver) OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, location uint64) error) { + b.onLocationCallbacks = append(b.onLocationCallbacks, fn) +} + +func (b *VoluntaryExitDeriver) Start(ctx context.Context) error { + if !b.cfg.Enabled { + b.log.Info("Voluntary exit deriver disabled") + + return nil + } + + b.log.Info("Voluntary exit deriver enabled") + + // Start our main loop + go b.run(ctx) + + return nil +} + +func (b *VoluntaryExitDeriver) Stop(ctx context.Context) error { + return nil +} + +func (b *VoluntaryExitDeriver) run(ctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 1 * time.Minute + + for { + select { + case <-ctx.Done(): + return + default: + operation := func() error { + time.Sleep(100 * time.Millisecond) + + // Get the next slot + location, err := b.iterator.Next(ctx) + if err != nil { + return err + } + + for _, fn := range b.onLocationCallbacks { + if errr := fn(ctx, location.GetEthV2BeaconBlockVoluntaryExit().GetSlot()); errr != nil { + b.log.WithError(errr).Error("Failed to send location") + } + } + + // Process the slot + events, err := b.processSlot(ctx, phase0.Slot(location.GetEthV2BeaconBlockVoluntaryExit().GetSlot())) + if err != nil { + b.log.WithError(err).Error("Failed to process slot") + + return err + } + + // Send the events + for _, event := range events { + for _, fn := range b.onEventCallbacks { + if err := fn(ctx, event); err != nil { + b.log.WithError(err).Error("Failed to send event") + } + } + } + + // Update our location + if err := b.iterator.UpdateLocation(ctx, location); err != nil { + return err + } + + bo.Reset() + + return nil + } + + if err := backoff.RetryNotify(operation, bo, func(err error, timer time.Duration) { + b.log.WithError(err).WithField("next_attempt", timer).Warn("Failed to process") + }); err != nil { + b.log.WithError(err).Warn("Failed to process") + } + } + } +} + +func (b *VoluntaryExitDeriver) processSlot(ctx context.Context, slot phase0.Slot) ([]*xatu.DecoratedEvent, error) { + // Get the block + block, err := b.beacon.GetBeaconBlock(ctx, xatuethv1.SlotAsString(slot)) + if err != nil { + return nil, errors.Wrapf(err, "failed to get beacon block for slot %d", slot) + } + + if block == nil { + return []*xatu.DecoratedEvent{}, nil + } + + blockIdentifier, err := GetBlockIdentifier(block, b.beacon.Metadata().Wallclock()) + if err != nil { + return nil, errors.Wrapf(err, "failed to get block identifier for slot %d", slot) + } + + events := []*xatu.DecoratedEvent{} + + exits, err := b.getVoluntaryExits(ctx, block) + if err != nil { + return nil, err + } + + for _, exit := range exits { + event, err := b.createEvent(ctx, exit, blockIdentifier) + if err != nil { + b.log.WithError(err).Error("Failed to create event") + + return nil, errors.Wrapf(err, "failed to create event for voluntary exit %s", exit.String()) + } + + events = append(events, event) + } + + return events, nil +} + +func (b *VoluntaryExitDeriver) getVoluntaryExits(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv1.SignedVoluntaryExitV2, error) { + exits := []*xatuethv1.SignedVoluntaryExitV2{} + + var voluntaryExits []*phase0.SignedVoluntaryExit + + switch block.Version { + case spec.DataVersionPhase0: + voluntaryExits = block.Phase0.Message.Body.VoluntaryExits + case spec.DataVersionAltair: + voluntaryExits = block.Altair.Message.Body.VoluntaryExits + case spec.DataVersionBellatrix: + voluntaryExits = block.Bellatrix.Message.Body.VoluntaryExits + case spec.DataVersionCapella: + voluntaryExits = block.Capella.Message.Body.VoluntaryExits + default: + return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) + } + + for _, exit := range voluntaryExits { + exits = append(exits, &xatuethv1.SignedVoluntaryExitV2{ + Message: &xatuethv1.VoluntaryExitV2{ + Epoch: wrapperspb.UInt64(uint64(exit.Message.Epoch)), + ValidatorIndex: wrapperspb.UInt64(uint64(exit.Message.ValidatorIndex)), + }, + Signature: exit.Signature.String(), + }) + } + + return exits, nil +} + +func (b *VoluntaryExitDeriver) createEvent(ctx context.Context, exit *xatuethv1.SignedVoluntaryExitV2, identifier *xatu.BlockIdentifier) (*xatu.DecoratedEvent, error) { + // Make a clone of the metadata + metadata, ok := proto.Clone(b.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV2BeaconBlockVoluntaryExit{ + EthV2BeaconBlockVoluntaryExit: exit, + }, + } + + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockVoluntaryExit{ + EthV2BeaconBlockVoluntaryExit: &xatu.ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData{ + Block: identifier, + }, + } + + return decoratedEvent, nil +} diff --git a/pkg/cannon/deriver/config.go b/pkg/cannon/deriver/config.go new file mode 100644 index 00000000..0f5f8ecd --- /dev/null +++ b/pkg/cannon/deriver/config.go @@ -0,0 +1,16 @@ +package deriver + +import v2 "github.com/ethpandaops/xatu/pkg/cannon/deriver/beacon/eth/v2" + +type Config struct { + AttesterSlashingConfig v2.AttesterSlashingDeriverConfig `yaml:"attesterSlashing"` + BLSToExecutionConfig v2.BLSToExecutionChangeDeriverConfig `yaml:"blsToExecutionChange"` + DepositConfig v2.DepositDeriverConfig `yaml:"deposit"` + ExecutionTransactionConfig v2.ExecutionTransactionDeriverConfig `yaml:"executionTransaction"` + ProposerSlashingConfig v2.ProposerSlashingDeriverConfig `yaml:"proposerSlashing"` + VoluntaryExitConfig v2.VoluntaryExitDeriverConfig `yaml:"voluntaryExit"` +} + +func (c *Config) Validate() error { + return nil +} diff --git a/pkg/cannon/deriver/event_deriver.go b/pkg/cannon/deriver/event_deriver.go new file mode 100644 index 00000000..bbf98250 --- /dev/null +++ b/pkg/cannon/deriver/event_deriver.go @@ -0,0 +1,26 @@ +package deriver + +import ( + "context" + + v2 "github.com/ethpandaops/xatu/pkg/cannon/deriver/beacon/eth/v2" + "github.com/ethpandaops/xatu/pkg/proto/xatu" +) + +type EventDeriver interface { + Start(ctx context.Context) error + Stop(ctx context.Context) error + Name() string + CannonType() xatu.CannonType + // Callbacks + OnEventDerived(ctx context.Context, fn func(ctx context.Context, event *xatu.DecoratedEvent) error) + OnLocationUpdated(ctx context.Context, fn func(ctx context.Context, loc uint64) error) +} + +// Ensure that derivers implements the EventDeriver interface +var _ EventDeriver = &v2.AttesterSlashingDeriver{} +var _ EventDeriver = &v2.ProposerSlashingDeriver{} +var _ EventDeriver = &v2.DepositDeriver{} +var _ EventDeriver = &v2.VoluntaryExitDeriver{} +var _ EventDeriver = &v2.ExecutionTransactionDeriver{} +var _ EventDeriver = &v2.BLSToExecutionChangeDeriver{} diff --git a/pkg/cannon/ethereum/beacon.go b/pkg/cannon/ethereum/beacon.go index 749a6fa1..6b29d3dd 100644 --- a/pkg/cannon/ethereum/beacon.go +++ b/pkg/cannon/ethereum/beacon.go @@ -6,11 +6,14 @@ import ( "sync" "time" + "github.com/attestantio/go-eth2-client/spec" "github.com/ethpandaops/beacon/pkg/beacon" "github.com/ethpandaops/xatu/pkg/cannon/ethereum/services" "github.com/go-co-op/gocron" + "github.com/jellydator/ttlcache/v3" "github.com/pkg/errors" "github.com/sirupsen/logrus" + "golang.org/x/sync/singleflight" ) type BeaconNode struct { @@ -22,12 +25,16 @@ type BeaconNode struct { services []services.Service onReadyCallbacks []func(ctx context.Context) error + + sfGroup *singleflight.Group + blockCache *ttlcache.Cache[string, *spec.VersionedSignedBeaconBlock] } func NewBeaconNode(ctx context.Context, name string, config *Config, log logrus.FieldLogger) (*BeaconNode, error) { opts := *beacon. DefaultOptions(). - EnableDefaultBeaconSubscription() + EnableDefaultBeaconSubscription(). + DisablePrometheusMetrics() opts.HealthCheck.Interval.Duration = time.Second * 3 opts.HealthCheck.SuccessfulResponses = 1 @@ -48,6 +55,11 @@ func NewBeaconNode(ctx context.Context, name string, config *Config, log logrus. log: log.WithField("module", "cannon/ethereum/beacon"), beacon: node, services: svcs, + blockCache: ttlcache.New( + ttlcache.WithTTL[string, *spec.VersionedSignedBeaconBlock](config.BlockCacheTTL.Duration), + ttlcache.WithCapacity[string, *spec.VersionedSignedBeaconBlock](config.BlockCacheSize), + ), + sfGroup: &singleflight.Group{}, }, nil } @@ -94,6 +106,8 @@ func (b *BeaconNode) Start(ctx context.Context) error { return err } + go b.blockCache.Start() + select { case err := <-errs: return err @@ -164,3 +178,30 @@ func (b *BeaconNode) Synced(ctx context.Context) error { return nil } + +// GetBeaconBlock returns a beacon block by its identifier. Blocks can be cached internally. +func (b *BeaconNode) GetBeaconBlock(ctx context.Context, identifier string) (*spec.VersionedSignedBeaconBlock, error) { + // Use singleflight to ensure we only make one request for a block at a time. + x, err, _ := b.sfGroup.Do(identifier, func() (interface{}, error) { + // Check the cache first. + if item := b.blockCache.Get(identifier); item != nil { + return item.Value(), nil + } + + // Not in the cache, so fetch it. + block, err := b.beacon.FetchBlock(ctx, identifier) + if err != nil { + return nil, err + } + + // Add it to the cache. + b.blockCache.Set(identifier, block, time.Hour) + + return block, nil + }) + if err != nil { + return nil, err + } + + return x.(*spec.VersionedSignedBeaconBlock), nil +} diff --git a/pkg/cannon/ethereum/config.go b/pkg/cannon/ethereum/config.go index ee6f3297..c16f15b2 100644 --- a/pkg/cannon/ethereum/config.go +++ b/pkg/cannon/ethereum/config.go @@ -1,6 +1,10 @@ package ethereum -import "errors" +import ( + "errors" + + "github.com/ethpandaops/beacon/pkg/human" +) type Config struct { // The address of the Beacon node to connect to @@ -8,6 +12,10 @@ type Config struct { // OverrideNetworkName is the name of the network to use for the sentry. // If not set, the network name will be retrieved from the beacon node. OverrideNetworkName string `yaml:"overrideNetworkName" default:""` + // BlockCacheSize is the number of blocks to cache. + BlockCacheSize uint64 `yaml:"blockCacheSize" default:"1000"` + // BlockCacheTTL is the time to live for blocks in the cache. + BlockCacheTTL human.Duration `yaml:"blockCacheTtl" default:"1h"` } func (c *Config) Validate() error { diff --git a/pkg/cannon/event/beacon/eth/v2/attester_slashing.go b/pkg/cannon/event/beacon/eth/v2/attester_slashing.go deleted file mode 100644 index 3218eb90..00000000 --- a/pkg/cannon/event/beacon/eth/v2/attester_slashing.go +++ /dev/null @@ -1,123 +0,0 @@ -package v2 - -import ( - "context" - - "github.com/attestantio/go-eth2-client/spec" - "github.com/attestantio/go-eth2-client/spec/phase0" - xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/google/uuid" - "github.com/pkg/errors" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -const ( - AttesterSlashingDeriverName = "BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING" -) - -type AttesterSlashingDeriver struct { - log logrus.FieldLogger -} - -func NewAttesterSlashingDeriver(log logrus.FieldLogger) *AttesterSlashingDeriver { - return &AttesterSlashingDeriver{ - log: log.WithField("module", "cannon/event/beacon/eth/v2/attester_slashing"), - } -} - -func (a *AttesterSlashingDeriver) Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) { - events := []*xatu.DecoratedEvent{} - - for _, slashing := range a.getAttesterSlashings(ctx, block) { - event, err := a.createEvent(ctx, metadata, slashing) - if err != nil { - a.log.WithError(err).Error("Failed to create event") - - return nil, errors.Wrapf(err, "failed to create event for attester slashing %s", slashing.String()) - } - - events = append(events, event) - } - - return events, nil -} - -func (a *AttesterSlashingDeriver) Name() string { - return AttesterSlashingDeriverName -} - -func (a *AttesterSlashingDeriver) getAttesterSlashings(ctx context.Context, block *spec.VersionedSignedBeaconBlock) []*xatuethv1.AttesterSlashingV2 { - slashings := []*xatuethv1.AttesterSlashingV2{} - - attesterSlashings, err := block.AttesterSlashings() - if err != nil { - a.log.WithError(err).Error("Failed to obtain attester slashings") - } - - for _, slashing := range attesterSlashings { - slashings = append(slashings, &xatuethv1.AttesterSlashingV2{ - Attestation_1: convertIndexedAttestation(slashing.Attestation1), - Attestation_2: convertIndexedAttestation(slashing.Attestation2), - }) - } - - return slashings -} - -func convertIndexedAttestation(attestation *phase0.IndexedAttestation) *xatuethv1.IndexedAttestationV2 { - indicies := []*wrapperspb.UInt64Value{} - - for _, index := range attestation.AttestingIndices { - indicies = append(indicies, &wrapperspb.UInt64Value{Value: index}) - } - - return &xatuethv1.IndexedAttestationV2{ - AttestingIndices: indicies, - Data: &xatuethv1.AttestationDataV2{ - Slot: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Slot)}, - Index: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Index)}, - BeaconBlockRoot: attestation.Data.BeaconBlockRoot.String(), - Source: &xatuethv1.CheckpointV2{ - Epoch: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Source.Epoch)}, - Root: attestation.Data.Source.Root.String(), - }, - Target: &xatuethv1.CheckpointV2{ - Epoch: &wrapperspb.UInt64Value{Value: uint64(attestation.Data.Target.Epoch)}, - Root: attestation.Data.Target.Root.String(), - }, - }, - Signature: attestation.Signature.String(), - } -} - -func (a *AttesterSlashingDeriver) createEvent(ctx context.Context, metadata *BeaconBlockMetadata, slashing *xatuethv1.AttesterSlashingV2) (*xatu.DecoratedEvent, error) { - decoratedEvent := &xatu.DecoratedEvent{ - Event: &xatu.Event{ - Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING, - DateTime: timestamppb.New(metadata.Now), - Id: uuid.New().String(), - }, - Meta: &xatu.Meta{ - Client: metadata.ClientMeta, - }, - Data: &xatu.DecoratedEvent_EthV2BeaconBlockAttesterSlashing{ - EthV2BeaconBlockAttesterSlashing: slashing, - }, - } - - blockIdentifier, err := metadata.BlockIdentifier() - if err != nil { - return nil, err - } - - decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockAttesterSlashing{ - EthV2BeaconBlockAttesterSlashing: &xatu.ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData{ - Block: blockIdentifier, - }, - } - - return decoratedEvent, nil -} diff --git a/pkg/cannon/event/beacon/eth/v2/beacon_block.go b/pkg/cannon/event/beacon/eth/v2/beacon_block.go deleted file mode 100644 index 3a89d0fe..00000000 --- a/pkg/cannon/event/beacon/eth/v2/beacon_block.go +++ /dev/null @@ -1,129 +0,0 @@ -package v2 - -import ( - "context" - "fmt" - "time" - - "github.com/attestantio/go-eth2-client/spec" - "github.com/ethpandaops/xatu/pkg/cannon/ethereum" - v1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -type BeaconBlockMetadata struct { - log logrus.FieldLogger - - Now time.Time - ClientMeta *xatu.ClientMeta - - block *spec.VersionedSignedBeaconBlock - - beacon *ethereum.BeaconNode - - processors []BeaconBlockEventDeriver -} - -func NewBeaconBlockMetadata(log logrus.FieldLogger, block *spec.VersionedSignedBeaconBlock, now time.Time, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta, processors []BeaconBlockEventDeriver) *BeaconBlockMetadata { - return &BeaconBlockMetadata{ - log: log, - Now: now, - block: block, - beacon: beacon, - ClientMeta: clientMeta, - processors: processors, - } -} - -func (e *BeaconBlockMetadata) BlockIdentifier() (*xatu.BlockIdentifier, error) { - if e.block == nil { - return nil, fmt.Errorf("block is nil") - } - - slotNum, err := e.block.Slot() - if err != nil { - e.log.WithError(err).Error("Failed to get block slot") - - return nil, err - } - - root, err := e.block.Root() - if err != nil { - e.log.WithError(err).Error("Failed to get block root") - - return nil, err - } - - slot := e.beacon.Metadata().Wallclock().Slots().FromNumber(uint64(slotNum)) - epoch := e.beacon.Metadata().Wallclock().Epochs().FromSlot(uint64(slotNum)) - - return &xatu.BlockIdentifier{ - Epoch: &xatu.EpochV2{ - Number: &wrapperspb.UInt64Value{Value: epoch.Number()}, - StartDateTime: timestamppb.New(epoch.TimeWindow().Start()), - }, - Slot: &xatu.SlotV2{ - Number: &wrapperspb.UInt64Value{Value: slot.Number()}, - StartDateTime: timestamppb.New(slot.TimeWindow().Start()), - }, - Root: v1.RootAsString(root), - Version: e.block.Version.String(), - }, nil -} - -func (e *BeaconBlockMetadata) Process(ctx context.Context) ([]*xatu.DecoratedEvent, error) { - if e.block == nil { - return nil, fmt.Errorf("block is nil") - } - - slot, err := e.block.Slot() - if err != nil { - e.log.WithError(err).Error("Failed to get block slot") - - return nil, nil - } - - root, err := e.block.Root() - if err != nil { - e.log.WithError(err).Error("Failed to get block root") - - return nil, err - } - - events := []*xatu.DecoratedEvent{} - - for _, processor := range e.processors { - evs, err := processor.Process(ctx, e, e.block) - if err != nil { - e.log.WithError(err).Error("Failed to process block") - - // Intentionally returning early here as we don't want to continue processing - // if one processor fails. - return nil, err - } - - e.log.WithFields(logrus.Fields{ - "processor": processor.Name(), - "events": len(evs), - "slot": slot, - "root": v1.RootAsString(root), - }).Info("Processor finished processing block") - - events = append(events, evs...) - } - - e.log.WithFields(logrus.Fields{ - "events": len(events), - "slot": slot, - "root": v1.RootAsString(root), - }).Info("Processed block") - - return events, nil -} - -func (e *BeaconBlockMetadata) Block() *spec.VersionedSignedBeaconBlock { - return e.block -} diff --git a/pkg/cannon/event/beacon/eth/v2/bls_to_execution_change.go b/pkg/cannon/event/beacon/eth/v2/bls_to_execution_change.go deleted file mode 100644 index 072eb6b9..00000000 --- a/pkg/cannon/event/beacon/eth/v2/bls_to_execution_change.go +++ /dev/null @@ -1,116 +0,0 @@ -package v2 - -import ( - "context" - "fmt" - - "github.com/attestantio/go-eth2-client/spec" - xatuethv2 "github.com/ethpandaops/xatu/pkg/proto/eth/v2" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/google/uuid" - "github.com/pkg/errors" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -const ( - BLSToExecutionChangeDeriverName = "BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE" -) - -type BLSToExecutionChangeDeriver struct { - log logrus.FieldLogger -} - -func NewBLSToExecutionChangeDeriver(log logrus.FieldLogger) *BLSToExecutionChangeDeriver { - return &BLSToExecutionChangeDeriver{ - log: log.WithField("module", "cannon/event/beacon/eth/v2/bls_to_execution_change"), - } -} - -func (b *BLSToExecutionChangeDeriver) Name() string { - return BLSToExecutionChangeDeriverName -} - -func (b *BLSToExecutionChangeDeriver) Filter(ctx context.Context) bool { - return false -} - -func (b *BLSToExecutionChangeDeriver) Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) { - events := []*xatu.DecoratedEvent{} - - changes, err := b.getBLSToExecutionChanges(ctx, block) - if err != nil { - return nil, err - } - - for _, change := range changes { - event, err := b.createEvent(ctx, metadata, change) - if err != nil { - b.log.WithError(err).Error("Failed to create event") - - return nil, errors.Wrapf(err, "failed to create event for BLS to execution change %s", change.String()) - } - - events = append(events, event) - } - - return events, nil -} - -func (b *BLSToExecutionChangeDeriver) getBLSToExecutionChanges(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv2.SignedBLSToExecutionChangeV2, error) { - changes := []*xatuethv2.SignedBLSToExecutionChangeV2{} - - switch block.Version { - case spec.DataVersionPhase0: - return changes, nil - case spec.DataVersionAltair: - return changes, nil - case spec.DataVersionBellatrix: - return changes, nil - case spec.DataVersionCapella: - for _, change := range block.Capella.Message.Body.BLSToExecutionChanges { - changes = append(changes, &xatuethv2.SignedBLSToExecutionChangeV2{ - Message: &xatuethv2.BLSToExecutionChangeV2{ - ValidatorIndex: wrapperspb.UInt64(uint64(change.Message.ValidatorIndex)), - FromBlsPubkey: change.Message.FromBLSPubkey.String(), - ToExecutionAddress: change.Message.ToExecutionAddress.String(), - }, - Signature: change.Signature.String(), - }) - } - default: - return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) - } - - return changes, nil -} - -func (b *BLSToExecutionChangeDeriver) createEvent(ctx context.Context, metadata *BeaconBlockMetadata, change *xatuethv2.SignedBLSToExecutionChangeV2) (*xatu.DecoratedEvent, error) { - decoratedEvent := &xatu.DecoratedEvent{ - Event: &xatu.Event{ - Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE, - DateTime: timestamppb.New(metadata.Now), - Id: uuid.New().String(), - }, - Meta: &xatu.Meta{ - Client: metadata.ClientMeta, - }, - Data: &xatu.DecoratedEvent_EthV2BeaconBlockBlsToExecutionChange{ - EthV2BeaconBlockBlsToExecutionChange: change, - }, - } - - blockIdentifier, err := metadata.BlockIdentifier() - if err != nil { - return nil, err - } - - decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockBlsToExecutionChange{ - EthV2BeaconBlockBlsToExecutionChange: &xatu.ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData{ - Block: blockIdentifier, - }, - } - - return decoratedEvent, nil -} diff --git a/pkg/cannon/event/beacon/eth/v2/deposit.go b/pkg/cannon/event/beacon/eth/v2/deposit.go deleted file mode 100644 index 0d37e039..00000000 --- a/pkg/cannon/event/beacon/eth/v2/deposit.go +++ /dev/null @@ -1,127 +0,0 @@ -package v2 - -import ( - "context" - "fmt" - - "github.com/attestantio/go-eth2-client/spec" - "github.com/attestantio/go-eth2-client/spec/phase0" - xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/google/uuid" - "github.com/pkg/errors" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -const ( - DepositDeriverName = "BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT" -) - -type DepositDeriver struct { - log logrus.FieldLogger -} - -func NewDepositDeriver(log logrus.FieldLogger) *DepositDeriver { - return &DepositDeriver{ - log: log.WithField("module", "cannon/event/beacon/eth/v2/withdrawal"), - } -} - -func (b *DepositDeriver) Name() string { - return DepositDeriverName -} - -func (b *DepositDeriver) Filter(ctx context.Context) bool { - return false -} - -func (b *DepositDeriver) Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) { - events := []*xatu.DecoratedEvent{} - - deposits, err := b.getDeposits(ctx, block) - if err != nil { - return nil, err - } - - for _, deposit := range deposits { - event, err := b.createEvent(ctx, metadata, deposit) - if err != nil { - b.log.WithError(err).Error("Failed to create event") - - return nil, errors.Wrapf(err, "failed to create event for deposit %s", deposit.String()) - } - - events = append(events, event) - } - - return events, nil -} - -func (b *DepositDeriver) getDeposits(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv1.DepositV2, error) { - exits := []*xatuethv1.DepositV2{} - - var deposits []*phase0.Deposit - - switch block.Version { - case spec.DataVersionPhase0: - deposits = block.Phase0.Message.Body.Deposits - case spec.DataVersionAltair: - deposits = block.Altair.Message.Body.Deposits - case spec.DataVersionBellatrix: - deposits = block.Bellatrix.Message.Body.Deposits - case spec.DataVersionCapella: - deposits = block.Capella.Message.Body.Deposits - default: - return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) - } - - for _, deposit := range deposits { - proof := []string{} - for _, p := range deposit.Proof { - proof = append(proof, fmt.Sprintf("0x%x", p)) - } - - exits = append(exits, &xatuethv1.DepositV2{ - Proof: proof, - Data: &xatuethv1.DepositV2_Data{ - Pubkey: deposit.Data.PublicKey.String(), - WithdrawalCredentials: fmt.Sprintf("0x%x", deposit.Data.WithdrawalCredentials), - Amount: wrapperspb.UInt64(uint64(deposit.Data.Amount)), - Signature: deposit.Data.Signature.String(), - }, - }) - } - - return exits, nil -} - -func (b *DepositDeriver) createEvent(ctx context.Context, metadata *BeaconBlockMetadata, deposit *xatuethv1.DepositV2) (*xatu.DecoratedEvent, error) { - decoratedEvent := &xatu.DecoratedEvent{ - Event: &xatu.Event{ - Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT, - DateTime: timestamppb.New(metadata.Now), - Id: uuid.New().String(), - }, - Meta: &xatu.Meta{ - Client: metadata.ClientMeta, - }, - Data: &xatu.DecoratedEvent_EthV2BeaconBlockDeposit{ - EthV2BeaconBlockDeposit: deposit, - }, - } - - blockIdentifier, err := metadata.BlockIdentifier() - if err != nil { - return nil, err - } - - decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockDeposit{ - EthV2BeaconBlockDeposit: &xatu.ClientMeta_AdditionalEthV2BeaconBlockDepositData{ - Block: blockIdentifier, - }, - } - - return decoratedEvent, nil -} diff --git a/pkg/cannon/event/beacon/eth/v2/event_deriver.go b/pkg/cannon/event/beacon/eth/v2/event_deriver.go deleted file mode 100644 index a3457be5..00000000 --- a/pkg/cannon/event/beacon/eth/v2/event_deriver.go +++ /dev/null @@ -1,13 +0,0 @@ -package v2 - -import ( - "context" - - "github.com/attestantio/go-eth2-client/spec" - "github.com/ethpandaops/xatu/pkg/proto/xatu" -) - -type BeaconBlockEventDeriver interface { - Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) - Name() string -} diff --git a/pkg/cannon/event/beacon/eth/v2/execution_transaction.go b/pkg/cannon/event/beacon/eth/v2/execution_transaction.go deleted file mode 100644 index 7d812c19..00000000 --- a/pkg/cannon/event/beacon/eth/v2/execution_transaction.go +++ /dev/null @@ -1,163 +0,0 @@ -package v2 - -import ( - "context" - "encoding/hex" - "fmt" - - "github.com/attestantio/go-eth2-client/spec" - "github.com/ethereum/go-ethereum/core/types" - xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/google/uuid" - "github.com/pkg/errors" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -type ExecutionTransactionDeriver struct { - log logrus.FieldLogger -} - -const ( - ExecutionTransactionDeriverName = "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION" -) - -func NewExecutionTransactionDeriver(log logrus.FieldLogger) *ExecutionTransactionDeriver { - return &ExecutionTransactionDeriver{ - log: log.WithField("module", "cannon/event/beacon/eth/v2/execution_transaction"), - } -} - -func (b *ExecutionTransactionDeriver) Name() string { - return ExecutionTransactionDeriverName -} - -func (b *ExecutionTransactionDeriver) Filter(ctx context.Context) bool { - return false -} - -func (b *ExecutionTransactionDeriver) Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) { - events := []*xatu.DecoratedEvent{} - - transactions, err := b.getExecutionTransactions(ctx, block, metadata) - if err != nil { - return nil, err - } - - for index, transaction := range transactions { - from, err := types.Sender(types.LatestSignerForChainID(transaction.ChainId()), transaction) - if err != nil { - return nil, fmt.Errorf("failed to get transaction sender: %v", err) - } - - gasPrice := transaction.GasPrice() - if gasPrice == nil { - return nil, fmt.Errorf("failed to get transaction gas price") - } - - value := transaction.Value() - if value == nil { - return nil, fmt.Errorf("failed to get transaction value") - } - - to := "" - - if transaction.To() != nil { - to = transaction.To().Hex() - } - - chainID := transaction.ChainId() - if chainID == nil { - return nil, fmt.Errorf("failed to get transaction chain ID") - } - - tx := &xatuethv1.Transaction{ - Nonce: wrapperspb.UInt64(transaction.Nonce()), - GasPrice: gasPrice.String(), - Gas: wrapperspb.UInt64(transaction.Gas()), - To: to, - From: from.Hex(), - Value: value.String(), - Input: hex.EncodeToString(transaction.Data()), - Hash: transaction.Hash().Hex(), - ChainId: chainID.String(), - Type: wrapperspb.UInt32(uint32(transaction.Type())), - } - - event, err := b.createEvent(ctx, metadata, tx, uint64(index)) - if err != nil { - b.log.WithError(err).Error("Failed to create event") - - return nil, errors.Wrapf(err, "failed to create event for execution transaction %s", transaction.Hash()) - } - - events = append(events, event) - } - - return events, nil -} - -func (b *ExecutionTransactionDeriver) getExecutionTransactions(ctx context.Context, block *spec.VersionedSignedBeaconBlock, metadata *BeaconBlockMetadata) ([]*types.Transaction, error) { - transactions := []*types.Transaction{} - - switch block.Version { - case spec.DataVersionPhase0: - return transactions, nil - case spec.DataVersionAltair: - return transactions, nil - case spec.DataVersionBellatrix: - for _, transaction := range block.Bellatrix.Message.Body.ExecutionPayload.Transactions { - ethTransaction := new(types.Transaction) - if err := ethTransaction.UnmarshalBinary(transaction); err != nil { - return nil, fmt.Errorf("failed to unmarshal transaction: %v", err) - } - - transactions = append(transactions, ethTransaction) - } - case spec.DataVersionCapella: - for _, transaction := range block.Capella.Message.Body.ExecutionPayload.Transactions { - ethTransaction := new(types.Transaction) - if err := ethTransaction.UnmarshalBinary(transaction); err != nil { - return nil, fmt.Errorf("failed to unmarshal transaction: %v", err) - } - - transactions = append(transactions, ethTransaction) - } - default: - return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) - } - - return transactions, nil -} - -func (b *ExecutionTransactionDeriver) createEvent(ctx context.Context, metadata *BeaconBlockMetadata, transaction *xatuethv1.Transaction, positionInBlock uint64) (*xatu.DecoratedEvent, error) { - decoratedEvent := &xatu.DecoratedEvent{ - Event: &xatu.Event{ - Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION, - DateTime: timestamppb.New(metadata.Now), - Id: uuid.New().String(), - }, - Meta: &xatu.Meta{ - Client: metadata.ClientMeta, - }, - Data: &xatu.DecoratedEvent_EthV2BeaconBlockExecutionTransaction{ - EthV2BeaconBlockExecutionTransaction: transaction, - }, - } - - blockIdentifier, err := metadata.BlockIdentifier() - if err != nil { - return nil, err - } - - decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockExecutionTransaction{ - EthV2BeaconBlockExecutionTransaction: &xatu.ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData{ - Block: blockIdentifier, - PositionInBlock: wrapperspb.UInt64(positionInBlock), - }, - } - - return decoratedEvent, nil -} diff --git a/pkg/cannon/event/beacon/eth/v2/proposer_slashing.go b/pkg/cannon/event/beacon/eth/v2/proposer_slashing.go deleted file mode 100644 index af4185af..00000000 --- a/pkg/cannon/event/beacon/eth/v2/proposer_slashing.go +++ /dev/null @@ -1,123 +0,0 @@ -package v2 - -import ( - "context" - - "github.com/attestantio/go-eth2-client/spec" - xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/google/uuid" - "github.com/pkg/errors" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -const ( - ProposerSlashingDeriverName = "BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING" -) - -type ProposerSlashingDeriver struct { - log logrus.FieldLogger -} - -func NewProposerSlashingDeriver(log logrus.FieldLogger) *ProposerSlashingDeriver { - return &ProposerSlashingDeriver{ - log: log.WithField("module", "cannon/event/beacon/eth/v2/proposer_slashing"), - } -} - -func (b *ProposerSlashingDeriver) Name() string { - return ProposerSlashingDeriverName -} - -func (b *ProposerSlashingDeriver) Filter(ctx context.Context) bool { - return false -} - -func (b *ProposerSlashingDeriver) Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) { - events := []*xatu.DecoratedEvent{} - - slashings, err := b.getProposerSlashings(ctx, block) - if err != nil { - return nil, err - } - - for _, slashing := range slashings { - event, err := b.createEvent(ctx, metadata, slashing) - if err != nil { - b.log.WithError(err).Error("Failed to create event") - - return nil, errors.Wrapf(err, "failed to create event for proposer slashing %s", slashing.String()) - } - - events = append(events, event) - } - - return events, nil -} - -func (b *ProposerSlashingDeriver) getProposerSlashings(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv1.ProposerSlashingV2, error) { - slashings := []*xatuethv1.ProposerSlashingV2{} - - blockSlashings, err := block.ProposerSlashings() - if err != nil { - return nil, err - } - - for _, slashing := range blockSlashings { - slashings = append(slashings, &xatuethv1.ProposerSlashingV2{ - SignedHeader_1: &xatuethv1.SignedBeaconBlockHeaderV2{ - Message: &xatuethv1.BeaconBlockHeaderV2{ - Slot: wrapperspb.UInt64(uint64(slashing.SignedHeader1.Message.Slot)), - ProposerIndex: wrapperspb.UInt64(uint64(slashing.SignedHeader1.Message.ProposerIndex)), - ParentRoot: slashing.SignedHeader1.Message.ParentRoot.String(), - StateRoot: slashing.SignedHeader1.Message.StateRoot.String(), - BodyRoot: slashing.SignedHeader1.Message.BodyRoot.String(), - }, - Signature: slashing.SignedHeader1.Signature.String(), - }, - SignedHeader_2: &xatuethv1.SignedBeaconBlockHeaderV2{ - Message: &xatuethv1.BeaconBlockHeaderV2{ - Slot: wrapperspb.UInt64(uint64(slashing.SignedHeader2.Message.Slot)), - ProposerIndex: wrapperspb.UInt64(uint64(slashing.SignedHeader2.Message.ProposerIndex)), - ParentRoot: slashing.SignedHeader2.Message.ParentRoot.String(), - StateRoot: slashing.SignedHeader2.Message.StateRoot.String(), - BodyRoot: slashing.SignedHeader2.Message.BodyRoot.String(), - }, - Signature: slashing.SignedHeader2.Signature.String(), - }, - }) - } - - return slashings, nil -} - -func (b *ProposerSlashingDeriver) createEvent(ctx context.Context, metadata *BeaconBlockMetadata, slashing *xatuethv1.ProposerSlashingV2) (*xatu.DecoratedEvent, error) { - decoratedEvent := &xatu.DecoratedEvent{ - Event: &xatu.Event{ - Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING, - DateTime: timestamppb.New(metadata.Now), - Id: uuid.New().String(), - }, - Meta: &xatu.Meta{ - Client: metadata.ClientMeta, - }, - Data: &xatu.DecoratedEvent_EthV2BeaconBlockProposerSlashing{ - EthV2BeaconBlockProposerSlashing: slashing, - }, - } - - blockIdentifier, err := metadata.BlockIdentifier() - if err != nil { - return nil, err - } - - decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockProposerSlashing{ - EthV2BeaconBlockProposerSlashing: &xatu.ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData{ - Block: blockIdentifier, - }, - } - - return decoratedEvent, nil -} diff --git a/pkg/cannon/event/beacon/eth/v2/voluntary_exit.go b/pkg/cannon/event/beacon/eth/v2/voluntary_exit.go deleted file mode 100644 index 832434b0..00000000 --- a/pkg/cannon/event/beacon/eth/v2/voluntary_exit.go +++ /dev/null @@ -1,120 +0,0 @@ -package v2 - -import ( - "context" - "fmt" - - "github.com/attestantio/go-eth2-client/spec" - "github.com/attestantio/go-eth2-client/spec/phase0" - xatuethv1 "github.com/ethpandaops/xatu/pkg/proto/eth/v1" - "github.com/ethpandaops/xatu/pkg/proto/xatu" - "github.com/google/uuid" - "github.com/pkg/errors" - "github.com/sirupsen/logrus" - "google.golang.org/protobuf/types/known/timestamppb" - "google.golang.org/protobuf/types/known/wrapperspb" -) - -const ( - VoluntaryExitDeriverName = "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT" -) - -type VoluntaryExitDeriver struct { - log logrus.FieldLogger -} - -func NewVoluntaryExitDeriver(log logrus.FieldLogger) *VoluntaryExitDeriver { - return &VoluntaryExitDeriver{ - log: log.WithField("module", "cannon/event/beacon/eth/v2/voluntary_exit"), - } -} - -func (b *VoluntaryExitDeriver) Name() string { - return VoluntaryExitDeriverName -} - -func (b *VoluntaryExitDeriver) Filter(ctx context.Context) bool { - return false -} - -func (b *VoluntaryExitDeriver) Process(ctx context.Context, metadata *BeaconBlockMetadata, block *spec.VersionedSignedBeaconBlock) ([]*xatu.DecoratedEvent, error) { - events := []*xatu.DecoratedEvent{} - - exits, err := b.getVoluntaryExits(ctx, block) - if err != nil { - return nil, err - } - - for _, exit := range exits { - event, err := b.createEvent(ctx, metadata, exit) - if err != nil { - b.log.WithError(err).Error("Failed to create event") - - return nil, errors.Wrapf(err, "failed to create event for voluntary exit %s", exit.String()) - } - - events = append(events, event) - } - - return events, nil -} - -func (b *VoluntaryExitDeriver) getVoluntaryExits(ctx context.Context, block *spec.VersionedSignedBeaconBlock) ([]*xatuethv1.SignedVoluntaryExitV2, error) { - exits := []*xatuethv1.SignedVoluntaryExitV2{} - - var voluntaryExits []*phase0.SignedVoluntaryExit - - switch block.Version { - case spec.DataVersionPhase0: - voluntaryExits = block.Phase0.Message.Body.VoluntaryExits - case spec.DataVersionAltair: - voluntaryExits = block.Altair.Message.Body.VoluntaryExits - case spec.DataVersionBellatrix: - voluntaryExits = block.Bellatrix.Message.Body.VoluntaryExits - case spec.DataVersionCapella: - voluntaryExits = block.Capella.Message.Body.VoluntaryExits - default: - return nil, fmt.Errorf("unsupported block version: %s", block.Version.String()) - } - - for _, exit := range voluntaryExits { - exits = append(exits, &xatuethv1.SignedVoluntaryExitV2{ - Message: &xatuethv1.VoluntaryExitV2{ - Epoch: wrapperspb.UInt64(uint64(exit.Message.Epoch)), - ValidatorIndex: wrapperspb.UInt64(uint64(exit.Message.ValidatorIndex)), - }, - Signature: exit.Signature.String(), - }) - } - - return exits, nil -} - -func (b *VoluntaryExitDeriver) createEvent(ctx context.Context, metadata *BeaconBlockMetadata, exit *xatuethv1.SignedVoluntaryExitV2) (*xatu.DecoratedEvent, error) { - decoratedEvent := &xatu.DecoratedEvent{ - Event: &xatu.Event{ - Name: xatu.Event_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT, - DateTime: timestamppb.New(metadata.Now), - Id: uuid.New().String(), - }, - Meta: &xatu.Meta{ - Client: metadata.ClientMeta, - }, - Data: &xatu.DecoratedEvent_EthV2BeaconBlockVoluntaryExit{ - EthV2BeaconBlockVoluntaryExit: exit, - }, - } - - blockIdentifier, err := metadata.BlockIdentifier() - if err != nil { - return nil, err - } - - decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV2BeaconBlockVoluntaryExit{ - EthV2BeaconBlockVoluntaryExit: &xatu.ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData{ - Block: blockIdentifier, - }, - } - - return decoratedEvent, nil -} diff --git a/pkg/cannon/iterator/iterator.go b/pkg/cannon/iterator/iterator.go new file mode 100644 index 00000000..e5fac36c --- /dev/null +++ b/pkg/cannon/iterator/iterator.go @@ -0,0 +1,17 @@ +package iterator + +import ( + "context" + "errors" + + "github.com/ethpandaops/xatu/pkg/proto/xatu" +) + +type Iterator interface { + UpdateLocation(ctx context.Context, location *xatu.CannonLocation) error + Next(ctx context.Context) (xatu.CannonLocation, error) +} + +var ( + ErrLocationUpToDate = errors.New("location up to date") +) diff --git a/pkg/cannon/iterator/slot_iterator.go b/pkg/cannon/iterator/slot_iterator.go new file mode 100644 index 00000000..84e5cc64 --- /dev/null +++ b/pkg/cannon/iterator/slot_iterator.go @@ -0,0 +1,162 @@ +package iterator + +import ( + "context" + "fmt" + "time" + + "github.com/attestantio/go-eth2-client/spec/phase0" + "github.com/ethpandaops/ethwallclock" + "github.com/ethpandaops/xatu/pkg/cannon/coordinator" + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/pkg/errors" + "github.com/sirupsen/logrus" +) + +type SlotIterator struct { + log logrus.FieldLogger + cannonType xatu.CannonType + coordinator coordinator.Client + wallclock *ethwallclock.EthereumBeaconChain + networkID string +} + +func NewSlotIterator(log logrus.FieldLogger, networkID string, cannonType xatu.CannonType, coordinatorClient *coordinator.Client, wallclock *ethwallclock.EthereumBeaconChain) *SlotIterator { + return &SlotIterator{ + log: log. + WithField("module", "cannon/iterator/slot_iterator"). + WithField("cannon_type", cannonType.String()), + networkID: networkID, + cannonType: cannonType, + coordinator: *coordinatorClient, + wallclock: wallclock, + } +} + +func (s *SlotIterator) UpdateLocation(ctx context.Context, location *xatu.CannonLocation) error { + return s.coordinator.UpsertCannonLocationRequest(ctx, location) +} + +func (s *SlotIterator) Next(ctx context.Context) (*xatu.CannonLocation, error) { + // Check where we are at from the coordinator + location, err := s.coordinator.GetCannonLocation(ctx, s.cannonType, s.networkID) + if err != nil { + return nil, errors.Wrap(err, "failed to get cannon location") + } + + // If location is empty we haven't started yet, start at slot 0 + if location == nil { + loc, errr := s.createLocationFromSlotNumber(0) + if errr != nil { + return nil, errors.Wrap(err, "failed to create location from slot number 0") + } + + return loc, nil + } + + // Calculate the current wallclock slot + headSlot, _, err := s.wallclock.Now() + if err != nil { + return nil, errors.Wrap(err, "failed to get current wallclock slot") + } + + locationSlot, err := s.getSlotNumberFromLocation(location) + if err != nil { + return nil, errors.Wrap(err, "failed to get slot number from location") + } + + // If the current wallclock slot is greater than the last slot we processed, return the next slot + if phase0.Slot(headSlot.Number()) > locationSlot { + loc, errr := s.createLocationFromSlotNumber(locationSlot + 1) + if errr != nil { + return nil, errors.Wrap(errr, fmt.Errorf("failed to create location from slot number: %d", locationSlot+1).Error()) + } + + return loc, nil + } + + // Sleep until the next slot + sleepTime := time.Until(headSlot.TimeWindow().Start()) + + if sleepTime.Milliseconds() > 0 { + s.log.WithField("sleep_time", sleepTime).Debug("sleeping until next slot") + + time.Sleep(sleepTime) + } + + loc, err := s.createLocationFromSlotNumber(phase0.Slot(headSlot.Number())) + if err != nil { + return nil, errors.Wrap(err, fmt.Errorf("failed to create location from slot number: %d", headSlot.Number()).Error()) + } + + return loc, nil +} + +func (s *SlotIterator) getSlotNumberFromLocation(location *xatu.CannonLocation) (phase0.Slot, error) { + switch location.Type { + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING: + return phase0.Slot(location.GetEthV2BeaconBlockAttesterSlashing().Slot), nil + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING: + return phase0.Slot(location.GetEthV2BeaconBlockProposerSlashing().Slot), nil + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE: + return phase0.Slot(location.GetEthV2BeaconBlockBlsToExecutionChange().Slot), nil + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION: + return phase0.Slot(location.GetEthV2BeaconBlockExecutionTransaction().Slot), nil + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT: + return phase0.Slot(location.GetEthV2BeaconBlockVoluntaryExit().Slot), nil + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT: + return phase0.Slot(location.GetEthV2BeaconBlockDeposit().Slot), nil + default: + return 0, errors.Errorf("unknown cannon type %s", location.Type) + } +} + +func (s *SlotIterator) createLocationFromSlotNumber(slot phase0.Slot) (*xatu.CannonLocation, error) { + location := &xatu.CannonLocation{ + NetworkId: s.networkID, + Type: s.cannonType, + } + + switch s.cannonType { + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING: + location.Data = &xatu.CannonLocation_EthV2BeaconBlockAttesterSlashing{ + EthV2BeaconBlockAttesterSlashing: &xatu.CannonLocationEthV2BeaconBlockAttesterSlashing{ + Slot: uint64(slot), + }, + } + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING: + location.Data = &xatu.CannonLocation_EthV2BeaconBlockProposerSlashing{ + EthV2BeaconBlockProposerSlashing: &xatu.CannonLocationEthV2BeaconBlockProposerSlashing{ + Slot: uint64(slot), + }, + } + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE: + location.Data = &xatu.CannonLocation_EthV2BeaconBlockBlsToExecutionChange{ + EthV2BeaconBlockBlsToExecutionChange: &xatu.CannonLocationEthV2BeaconBlockBlsToExecutionChange{ + Slot: uint64(slot), + }, + } + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION: + location.Data = &xatu.CannonLocation_EthV2BeaconBlockExecutionTransaction{ + EthV2BeaconBlockExecutionTransaction: &xatu.CannonLocationEthV2BeaconBlockExecutionTransaction{ + Slot: uint64(slot), + }, + } + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT: + location.Data = &xatu.CannonLocation_EthV2BeaconBlockVoluntaryExit{ + EthV2BeaconBlockVoluntaryExit: &xatu.CannonLocationEthV2BeaconBlockVoluntaryExit{ + Slot: uint64(slot), + }, + } + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT: + location.Data = &xatu.CannonLocation_EthV2BeaconBlockDeposit{ + EthV2BeaconBlockDeposit: &xatu.CannonLocationEthV2BeaconBlockDeposit{ + Slot: uint64(slot), + }, + } + default: + return location, errors.Errorf("unknown cannon type %s", location.Type) + } + + return location, nil +} diff --git a/pkg/cannon/metrics.go b/pkg/cannon/metrics.go index dfd86d53..f0e3ea9a 100644 --- a/pkg/cannon/metrics.go +++ b/pkg/cannon/metrics.go @@ -1,9 +1,13 @@ package cannon -import "github.com/prometheus/client_golang/prometheus" +import ( + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/prometheus/client_golang/prometheus" +) type Metrics struct { decoratedEventTotal *prometheus.CounterVec + deriverLocation *prometheus.GaugeVec } func NewMetrics(namespace string) *Metrics { @@ -11,15 +15,25 @@ func NewMetrics(namespace string) *Metrics { decoratedEventTotal: prometheus.NewCounterVec(prometheus.CounterOpts{ Namespace: namespace, Name: "decorated_event_total", - Help: "Total number of decorated events received", - }, []string{"type", "network_id"}), + Help: "Total number of decorated events created by the cannon", + }, []string{"type"}), + deriverLocation: prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: namespace, + Name: "deriver_location", + Help: "Location of the cannon event deriver", + }, []string{"type"}), } prometheus.MustRegister(m.decoratedEventTotal) + prometheus.MustRegister(m.deriverLocation) return m } -func (m *Metrics) AddDecoratedEvent(count int, eventType, networkID string) { - m.decoratedEventTotal.WithLabelValues(eventType, networkID).Add(float64(count)) +func (m *Metrics) AddDecoratedEvent(count int, eventType xatu.CannonType) { + m.decoratedEventTotal.WithLabelValues(eventType.String()).Add(float64(count)) +} + +func (m *Metrics) SetDeriverLocation(location uint64, eventType xatu.CannonType) { + m.deriverLocation.WithLabelValues(eventType.String()).Set(float64(location)) } diff --git a/pkg/networks/network.go b/pkg/networks/network.go index a285c9d9..c2aa4ee6 100644 --- a/pkg/networks/network.go +++ b/pkg/networks/network.go @@ -13,6 +13,7 @@ var ( NetworkNameMainnet NetworkName = "mainnet" NetworkNameGoerli NetworkName = "goerli" NetworkNameSepolia NetworkName = "sepolia" + NetworkNameHolesky NetworkName = "holesky" ) var NetworkGenesisRoots = map[string]uint64{ @@ -25,6 +26,7 @@ var NetworkIds = map[uint64]NetworkName{ 1: NetworkNameMainnet, 5: NetworkNameGoerli, 11155111: NetworkNameSepolia, + 17000: NetworkNameHolesky, } func DeriveFromGenesisRoot(genesisRoot string) *Network { diff --git a/pkg/proto/xatu/coordinator.pb.go b/pkg/proto/xatu/coordinator.pb.go index a36a4abe..1ebaec46 100644 --- a/pkg/proto/xatu/coordinator.pb.go +++ b/pkg/proto/xatu/coordinator.pb.go @@ -23,9 +23,12 @@ const ( type CannonType int32 const ( - CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT CannonType = 0 - CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING CannonType = 1 - CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT CannonType = 2 + CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT CannonType = 0 + CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING CannonType = 1 + CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT CannonType = 2 + CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING CannonType = 3 + CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE CannonType = 4 + CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION CannonType = 5 ) // Enum value maps for CannonType. @@ -34,11 +37,17 @@ var ( 0: "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT", 1: "BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING", 2: "BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT", + 3: "BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING", + 4: "BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE", + 5: "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION", } CannonType_value = map[string]int32{ - "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT": 0, - "BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING": 1, - "BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT": 2, + "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT": 0, + "BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING": 1, + "BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT": 2, + "BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING": 3, + "BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE": 4, + "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION": 5, } ) @@ -884,6 +893,147 @@ func (x *CannonLocationEthV2BeaconBlockDeposit) GetSlot() uint64 { return 0 } +type CannonLocationEthV2BeaconBlockAttesterSlashing struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Slot uint64 `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty"` +} + +func (x *CannonLocationEthV2BeaconBlockAttesterSlashing) Reset() { + *x = CannonLocationEthV2BeaconBlockAttesterSlashing{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CannonLocationEthV2BeaconBlockAttesterSlashing) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CannonLocationEthV2BeaconBlockAttesterSlashing) ProtoMessage() {} + +func (x *CannonLocationEthV2BeaconBlockAttesterSlashing) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[15] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CannonLocationEthV2BeaconBlockAttesterSlashing.ProtoReflect.Descriptor instead. +func (*CannonLocationEthV2BeaconBlockAttesterSlashing) Descriptor() ([]byte, []int) { + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{15} +} + +func (x *CannonLocationEthV2BeaconBlockAttesterSlashing) GetSlot() uint64 { + if x != nil { + return x.Slot + } + return 0 +} + +type CannonLocationEthV2BeaconBlockBlsToExecutionChange struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Slot uint64 `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty"` +} + +func (x *CannonLocationEthV2BeaconBlockBlsToExecutionChange) Reset() { + *x = CannonLocationEthV2BeaconBlockBlsToExecutionChange{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CannonLocationEthV2BeaconBlockBlsToExecutionChange) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CannonLocationEthV2BeaconBlockBlsToExecutionChange) ProtoMessage() {} + +func (x *CannonLocationEthV2BeaconBlockBlsToExecutionChange) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[16] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CannonLocationEthV2BeaconBlockBlsToExecutionChange.ProtoReflect.Descriptor instead. +func (*CannonLocationEthV2BeaconBlockBlsToExecutionChange) Descriptor() ([]byte, []int) { + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{16} +} + +func (x *CannonLocationEthV2BeaconBlockBlsToExecutionChange) GetSlot() uint64 { + if x != nil { + return x.Slot + } + return 0 +} + +type CannonLocationEthV2BeaconBlockExecutionTransaction struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Slot uint64 `protobuf:"varint,1,opt,name=slot,proto3" json:"slot,omitempty"` +} + +func (x *CannonLocationEthV2BeaconBlockExecutionTransaction) Reset() { + *x = CannonLocationEthV2BeaconBlockExecutionTransaction{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CannonLocationEthV2BeaconBlockExecutionTransaction) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CannonLocationEthV2BeaconBlockExecutionTransaction) ProtoMessage() {} + +func (x *CannonLocationEthV2BeaconBlockExecutionTransaction) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[17] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use CannonLocationEthV2BeaconBlockExecutionTransaction.ProtoReflect.Descriptor instead. +func (*CannonLocationEthV2BeaconBlockExecutionTransaction) Descriptor() ([]byte, []int) { + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{17} +} + +func (x *CannonLocationEthV2BeaconBlockExecutionTransaction) GetSlot() uint64 { + if x != nil { + return x.Slot + } + return 0 +} + type CannonLocation struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -896,13 +1046,16 @@ type CannonLocation struct { // *CannonLocation_EthV2BeaconBlockVoluntaryExit // *CannonLocation_EthV2BeaconBlockProposerSlashing // *CannonLocation_EthV2BeaconBlockDeposit + // *CannonLocation_EthV2BeaconBlockAttesterSlashing + // *CannonLocation_EthV2BeaconBlockBlsToExecutionChange + // *CannonLocation_EthV2BeaconBlockExecutionTransaction Data isCannonLocation_Data `protobuf_oneof:"Data"` } func (x *CannonLocation) Reset() { *x = CannonLocation{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[15] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -915,7 +1068,7 @@ func (x *CannonLocation) String() string { func (*CannonLocation) ProtoMessage() {} func (x *CannonLocation) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[15] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -928,7 +1081,7 @@ func (x *CannonLocation) ProtoReflect() protoreflect.Message { // Deprecated: Use CannonLocation.ProtoReflect.Descriptor instead. func (*CannonLocation) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{15} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{18} } func (x *CannonLocation) GetNetworkId() string { @@ -973,6 +1126,27 @@ func (x *CannonLocation) GetEthV2BeaconBlockDeposit() *CannonLocationEthV2Beacon return nil } +func (x *CannonLocation) GetEthV2BeaconBlockAttesterSlashing() *CannonLocationEthV2BeaconBlockAttesterSlashing { + if x, ok := x.GetData().(*CannonLocation_EthV2BeaconBlockAttesterSlashing); ok { + return x.EthV2BeaconBlockAttesterSlashing + } + return nil +} + +func (x *CannonLocation) GetEthV2BeaconBlockBlsToExecutionChange() *CannonLocationEthV2BeaconBlockBlsToExecutionChange { + if x, ok := x.GetData().(*CannonLocation_EthV2BeaconBlockBlsToExecutionChange); ok { + return x.EthV2BeaconBlockBlsToExecutionChange + } + return nil +} + +func (x *CannonLocation) GetEthV2BeaconBlockExecutionTransaction() *CannonLocationEthV2BeaconBlockExecutionTransaction { + if x, ok := x.GetData().(*CannonLocation_EthV2BeaconBlockExecutionTransaction); ok { + return x.EthV2BeaconBlockExecutionTransaction + } + return nil +} + type isCannonLocation_Data interface { isCannonLocation_Data() } @@ -989,12 +1163,30 @@ type CannonLocation_EthV2BeaconBlockDeposit struct { EthV2BeaconBlockDeposit *CannonLocationEthV2BeaconBlockDeposit `protobuf:"bytes,5,opt,name=eth_v2_beacon_block_deposit,json=BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT,proto3,oneof"` } +type CannonLocation_EthV2BeaconBlockAttesterSlashing struct { + EthV2BeaconBlockAttesterSlashing *CannonLocationEthV2BeaconBlockAttesterSlashing `protobuf:"bytes,6,opt,name=eth_v2_beacon_block_attester_slashing,json=BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING,proto3,oneof"` +} + +type CannonLocation_EthV2BeaconBlockBlsToExecutionChange struct { + EthV2BeaconBlockBlsToExecutionChange *CannonLocationEthV2BeaconBlockBlsToExecutionChange `protobuf:"bytes,7,opt,name=eth_v2_beacon_block_bls_to_execution_change,json=BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE,proto3,oneof"` +} + +type CannonLocation_EthV2BeaconBlockExecutionTransaction struct { + EthV2BeaconBlockExecutionTransaction *CannonLocationEthV2BeaconBlockExecutionTransaction `protobuf:"bytes,8,opt,name=eth_v2_beacon_block_execution_transaction,json=BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION,proto3,oneof"` +} + func (*CannonLocation_EthV2BeaconBlockVoluntaryExit) isCannonLocation_Data() {} func (*CannonLocation_EthV2BeaconBlockProposerSlashing) isCannonLocation_Data() {} func (*CannonLocation_EthV2BeaconBlockDeposit) isCannonLocation_Data() {} +func (*CannonLocation_EthV2BeaconBlockAttesterSlashing) isCannonLocation_Data() {} + +func (*CannonLocation_EthV2BeaconBlockBlsToExecutionChange) isCannonLocation_Data() {} + +func (*CannonLocation_EthV2BeaconBlockExecutionTransaction) isCannonLocation_Data() {} + type GetCannonLocationRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1007,7 +1199,7 @@ type GetCannonLocationRequest struct { func (x *GetCannonLocationRequest) Reset() { *x = GetCannonLocationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[16] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1020,7 +1212,7 @@ func (x *GetCannonLocationRequest) String() string { func (*GetCannonLocationRequest) ProtoMessage() {} func (x *GetCannonLocationRequest) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[16] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1033,7 +1225,7 @@ func (x *GetCannonLocationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCannonLocationRequest.ProtoReflect.Descriptor instead. func (*GetCannonLocationRequest) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{16} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{19} } func (x *GetCannonLocationRequest) GetNetworkId() string { @@ -1061,7 +1253,7 @@ type GetCannonLocationResponse struct { func (x *GetCannonLocationResponse) Reset() { *x = GetCannonLocationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[17] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1074,7 +1266,7 @@ func (x *GetCannonLocationResponse) String() string { func (*GetCannonLocationResponse) ProtoMessage() {} func (x *GetCannonLocationResponse) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[17] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1087,7 +1279,7 @@ func (x *GetCannonLocationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetCannonLocationResponse.ProtoReflect.Descriptor instead. func (*GetCannonLocationResponse) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{17} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{20} } func (x *GetCannonLocationResponse) GetLocation() *CannonLocation { @@ -1108,7 +1300,7 @@ type UpsertCannonLocationRequest struct { func (x *UpsertCannonLocationRequest) Reset() { *x = UpsertCannonLocationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[18] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1121,7 +1313,7 @@ func (x *UpsertCannonLocationRequest) String() string { func (*UpsertCannonLocationRequest) ProtoMessage() {} func (x *UpsertCannonLocationRequest) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[18] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1134,7 +1326,7 @@ func (x *UpsertCannonLocationRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpsertCannonLocationRequest.ProtoReflect.Descriptor instead. func (*UpsertCannonLocationRequest) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{18} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{21} } func (x *UpsertCannonLocationRequest) GetLocation() *CannonLocation { @@ -1153,7 +1345,7 @@ type UpsertCannonLocationResponse struct { func (x *UpsertCannonLocationResponse) Reset() { *x = UpsertCannonLocationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[19] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1166,7 +1358,7 @@ func (x *UpsertCannonLocationResponse) String() string { func (*UpsertCannonLocationResponse) ProtoMessage() {} func (x *UpsertCannonLocationResponse) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[19] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1179,7 +1371,7 @@ func (x *UpsertCannonLocationResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UpsertCannonLocationResponse.ProtoReflect.Descriptor instead. func (*UpsertCannonLocationResponse) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{19} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{22} } type ExecutionNodeStatus_Capability struct { @@ -1194,7 +1386,7 @@ type ExecutionNodeStatus_Capability struct { func (x *ExecutionNodeStatus_Capability) Reset() { *x = ExecutionNodeStatus_Capability{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[20] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1207,7 +1399,7 @@ func (x *ExecutionNodeStatus_Capability) String() string { func (*ExecutionNodeStatus_Capability) ProtoMessage() {} func (x *ExecutionNodeStatus_Capability) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[20] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1249,7 +1441,7 @@ type ExecutionNodeStatus_ForkID struct { func (x *ExecutionNodeStatus_ForkID) Reset() { *x = ExecutionNodeStatus_ForkID{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[21] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1262,7 +1454,7 @@ func (x *ExecutionNodeStatus_ForkID) String() string { func (*ExecutionNodeStatus_ForkID) ProtoMessage() {} func (x *ExecutionNodeStatus_ForkID) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[21] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1404,67 +1596,122 @@ var file_pkg_proto_xatu_coordinator_proto_rawDesc = []byte{ 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x6c, 0x6f, - 0x74, 0x22, 0x84, 0x04, 0x0a, 0x0e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, - 0x6b, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x54, - 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x22, 0x65, 0x74, - 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, - 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, - 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x6f, 0x6c, 0x75, - 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x48, 0x00, 0x52, 0x2d, 0x42, 0x45, 0x41, - 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, - 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x12, 0x97, 0x01, 0x0a, 0x25, 0x65, + 0x74, 0x22, 0x44, 0x0a, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, + 0x6f, 0x63, 0x6b, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, + 0x69, 0x6e, 0x67, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x22, 0x48, 0x0a, 0x32, 0x43, 0x61, 0x6e, 0x6e, 0x6f, + 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, + 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x6c, 0x73, 0x54, 0x6f, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x6c, 0x6f, + 0x74, 0x22, 0x48, 0x0a, 0x32, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, + 0x6f, 0x63, 0x6b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x22, 0xee, 0x07, 0x0a, 0x0e, + 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1d, + 0x0a, 0x0a, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x49, 0x64, 0x12, 0x24, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x10, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x22, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, + 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x6f, 0x6c, 0x75, + 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x31, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, + 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, + 0x78, 0x69, 0x74, 0x48, 0x00, 0x52, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, + 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, + 0x45, 0x58, 0x49, 0x54, 0x12, 0x97, 0x01, 0x0a, 0x25, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, + 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, + 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, + 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, + 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, + 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x30, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x50, 0x52, 0x4f, + 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x12, 0x7a, + 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, + 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, + 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, + 0x48, 0x00, 0x52, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, + 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x12, 0x97, 0x01, 0x0a, 0x25, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, - 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, - 0x68, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x78, 0x61, 0x74, + 0x63, 0x6b, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, + 0x68, 0x69, 0x6e, 0x67, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, - 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, + 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x48, 0x00, 0x52, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, - 0x43, 0x4b, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, - 0x48, 0x49, 0x4e, 0x47, 0x12, 0x7a, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x70, 0x6f, - 0x73, 0x69, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, - 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, - 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x48, 0x00, 0x52, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, - 0x42, 0x06, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x61, 0x22, 0x5f, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x43, - 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, - 0x6b, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x54, - 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x4d, 0x0a, 0x19, 0x47, 0x65, 0x74, - 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, - 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, - 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x4f, 0x0a, 0x1b, 0x55, 0x70, 0x73, 0x65, - 0x72, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x1e, 0x0a, 0x1c, 0x55, 0x70, 0x73, - 0x65, 0x72, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0xa1, 0x01, 0x0a, 0x0a, 0x43, 0x61, - 0x6e, 0x6e, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x31, 0x0a, 0x2d, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, - 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, - 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x10, 0x00, 0x12, 0x34, 0x0a, 0x30, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, - 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x50, 0x52, - 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, - 0x01, 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, + 0x43, 0x4b, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, + 0x48, 0x49, 0x4e, 0x47, 0x12, 0xa7, 0x01, 0x0a, 0x2b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, + 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x62, 0x6c, 0x73, + 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x68, + 0x61, 0x6e, 0x67, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x78, 0x61, 0x74, + 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x42, 0x6c, 0x73, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x68, + 0x61, 0x6e, 0x67, 0x65, 0x48, 0x00, 0x52, 0x36, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, + 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, + 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x42, 0x4c, 0x53, 0x5f, 0x54, 0x4f, 0x5f, 0x45, 0x58, + 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x12, 0xa3, + 0x01, 0x0a, 0x29, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, + 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, + 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, + 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x34, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, + 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x45, + 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, + 0x54, 0x49, 0x4f, 0x4e, 0x42, 0x06, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x61, 0x22, 0x5f, 0x0a, 0x18, + 0x47, 0x65, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x6e, 0x65, 0x74, 0x77, + 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x65, + 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, + 0x6e, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x22, 0x4d, 0x0a, + 0x19, 0x47, 0x65, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x08, 0x6c, 0x6f, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x4f, 0x0a, 0x1b, + 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x08, 0x6c, + 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x1e, 0x0a, + 0x1c, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0xcd, 0x02, + 0x0a, 0x0a, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x31, 0x0a, 0x2d, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, + 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, + 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x10, 0x00, 0x12, + 0x34, 0x0a, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, + 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, + 0x4b, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, + 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x10, + 0x02, 0x12, 0x34, 0x0a, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, - 0x4f, 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x10, 0x02, 0x32, 0x8a, 0x06, + 0x4f, 0x43, 0x4b, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, + 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x12, 0x3a, 0x0a, 0x36, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, + 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x42, 0x4c, 0x53, 0x5f, 0x54, 0x4f, + 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, + 0x45, 0x10, 0x04, 0x12, 0x38, 0x0a, 0x34, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, + 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, + 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x05, 0x32, 0x8a, 0x06, 0x0a, 0x0b, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x56, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x12, 0x1e, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, @@ -1532,63 +1779,69 @@ func file_pkg_proto_xatu_coordinator_proto_rawDescGZIP() []byte { } var file_pkg_proto_xatu_coordinator_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_pkg_proto_xatu_coordinator_proto_msgTypes = make([]protoimpl.MessageInfo, 22) +var file_pkg_proto_xatu_coordinator_proto_msgTypes = make([]protoimpl.MessageInfo, 25) var file_pkg_proto_xatu_coordinator_proto_goTypes = []interface{}{ - (CannonType)(0), // 0: xatu.CannonType - (*CreateNodeRecordsRequest)(nil), // 1: xatu.CreateNodeRecordsRequest - (*CreateNodeRecordsResponse)(nil), // 2: xatu.CreateNodeRecordsResponse - (*ListStalledExecutionNodeRecordsRequest)(nil), // 3: xatu.ListStalledExecutionNodeRecordsRequest - (*ListStalledExecutionNodeRecordsResponse)(nil), // 4: xatu.ListStalledExecutionNodeRecordsResponse - (*ExecutionNodeStatus)(nil), // 5: xatu.ExecutionNodeStatus - (*CreateExecutionNodeRecordStatusRequest)(nil), // 6: xatu.CreateExecutionNodeRecordStatusRequest - (*CreateExecutionNodeRecordStatusResponse)(nil), // 7: xatu.CreateExecutionNodeRecordStatusResponse - (*CoordinatedNodeRecord)(nil), // 8: xatu.CoordinatedNodeRecord - (*CoordinateExecutionNodeRecordsRequest)(nil), // 9: xatu.CoordinateExecutionNodeRecordsRequest - (*CoordinateExecutionNodeRecordsResponse)(nil), // 10: xatu.CoordinateExecutionNodeRecordsResponse - (*GetDiscoveryNodeRecordRequest)(nil), // 11: xatu.GetDiscoveryNodeRecordRequest - (*GetDiscoveryNodeRecordResponse)(nil), // 12: xatu.GetDiscoveryNodeRecordResponse - (*CannonLocationEthV2BeaconBlockVoluntaryExit)(nil), // 13: xatu.CannonLocationEthV2BeaconBlockVoluntaryExit - (*CannonLocationEthV2BeaconBlockProposerSlashing)(nil), // 14: xatu.CannonLocationEthV2BeaconBlockProposerSlashing - (*CannonLocationEthV2BeaconBlockDeposit)(nil), // 15: xatu.CannonLocationEthV2BeaconBlockDeposit - (*CannonLocation)(nil), // 16: xatu.CannonLocation - (*GetCannonLocationRequest)(nil), // 17: xatu.GetCannonLocationRequest - (*GetCannonLocationResponse)(nil), // 18: xatu.GetCannonLocationResponse - (*UpsertCannonLocationRequest)(nil), // 19: xatu.UpsertCannonLocationRequest - (*UpsertCannonLocationResponse)(nil), // 20: xatu.UpsertCannonLocationResponse - (*ExecutionNodeStatus_Capability)(nil), // 21: xatu.ExecutionNodeStatus.Capability - (*ExecutionNodeStatus_ForkID)(nil), // 22: xatu.ExecutionNodeStatus.ForkID + (CannonType)(0), // 0: xatu.CannonType + (*CreateNodeRecordsRequest)(nil), // 1: xatu.CreateNodeRecordsRequest + (*CreateNodeRecordsResponse)(nil), // 2: xatu.CreateNodeRecordsResponse + (*ListStalledExecutionNodeRecordsRequest)(nil), // 3: xatu.ListStalledExecutionNodeRecordsRequest + (*ListStalledExecutionNodeRecordsResponse)(nil), // 4: xatu.ListStalledExecutionNodeRecordsResponse + (*ExecutionNodeStatus)(nil), // 5: xatu.ExecutionNodeStatus + (*CreateExecutionNodeRecordStatusRequest)(nil), // 6: xatu.CreateExecutionNodeRecordStatusRequest + (*CreateExecutionNodeRecordStatusResponse)(nil), // 7: xatu.CreateExecutionNodeRecordStatusResponse + (*CoordinatedNodeRecord)(nil), // 8: xatu.CoordinatedNodeRecord + (*CoordinateExecutionNodeRecordsRequest)(nil), // 9: xatu.CoordinateExecutionNodeRecordsRequest + (*CoordinateExecutionNodeRecordsResponse)(nil), // 10: xatu.CoordinateExecutionNodeRecordsResponse + (*GetDiscoveryNodeRecordRequest)(nil), // 11: xatu.GetDiscoveryNodeRecordRequest + (*GetDiscoveryNodeRecordResponse)(nil), // 12: xatu.GetDiscoveryNodeRecordResponse + (*CannonLocationEthV2BeaconBlockVoluntaryExit)(nil), // 13: xatu.CannonLocationEthV2BeaconBlockVoluntaryExit + (*CannonLocationEthV2BeaconBlockProposerSlashing)(nil), // 14: xatu.CannonLocationEthV2BeaconBlockProposerSlashing + (*CannonLocationEthV2BeaconBlockDeposit)(nil), // 15: xatu.CannonLocationEthV2BeaconBlockDeposit + (*CannonLocationEthV2BeaconBlockAttesterSlashing)(nil), // 16: xatu.CannonLocationEthV2BeaconBlockAttesterSlashing + (*CannonLocationEthV2BeaconBlockBlsToExecutionChange)(nil), // 17: xatu.CannonLocationEthV2BeaconBlockBlsToExecutionChange + (*CannonLocationEthV2BeaconBlockExecutionTransaction)(nil), // 18: xatu.CannonLocationEthV2BeaconBlockExecutionTransaction + (*CannonLocation)(nil), // 19: xatu.CannonLocation + (*GetCannonLocationRequest)(nil), // 20: xatu.GetCannonLocationRequest + (*GetCannonLocationResponse)(nil), // 21: xatu.GetCannonLocationResponse + (*UpsertCannonLocationRequest)(nil), // 22: xatu.UpsertCannonLocationRequest + (*UpsertCannonLocationResponse)(nil), // 23: xatu.UpsertCannonLocationResponse + (*ExecutionNodeStatus_Capability)(nil), // 24: xatu.ExecutionNodeStatus.Capability + (*ExecutionNodeStatus_ForkID)(nil), // 25: xatu.ExecutionNodeStatus.ForkID } var file_pkg_proto_xatu_coordinator_proto_depIdxs = []int32{ - 21, // 0: xatu.ExecutionNodeStatus.capabilities:type_name -> xatu.ExecutionNodeStatus.Capability - 22, // 1: xatu.ExecutionNodeStatus.fork_id:type_name -> xatu.ExecutionNodeStatus.ForkID + 24, // 0: xatu.ExecutionNodeStatus.capabilities:type_name -> xatu.ExecutionNodeStatus.Capability + 25, // 1: xatu.ExecutionNodeStatus.fork_id:type_name -> xatu.ExecutionNodeStatus.ForkID 5, // 2: xatu.CreateExecutionNodeRecordStatusRequest.status:type_name -> xatu.ExecutionNodeStatus 8, // 3: xatu.CoordinateExecutionNodeRecordsRequest.node_records:type_name -> xatu.CoordinatedNodeRecord 0, // 4: xatu.CannonLocation.type:type_name -> xatu.CannonType 13, // 5: xatu.CannonLocation.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.CannonLocationEthV2BeaconBlockVoluntaryExit 14, // 6: xatu.CannonLocation.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.CannonLocationEthV2BeaconBlockProposerSlashing 15, // 7: xatu.CannonLocation.eth_v2_beacon_block_deposit:type_name -> xatu.CannonLocationEthV2BeaconBlockDeposit - 0, // 8: xatu.GetCannonLocationRequest.type:type_name -> xatu.CannonType - 16, // 9: xatu.GetCannonLocationResponse.location:type_name -> xatu.CannonLocation - 16, // 10: xatu.UpsertCannonLocationRequest.location:type_name -> xatu.CannonLocation - 1, // 11: xatu.Coordinator.CreateNodeRecords:input_type -> xatu.CreateNodeRecordsRequest - 3, // 12: xatu.Coordinator.ListStalledExecutionNodeRecords:input_type -> xatu.ListStalledExecutionNodeRecordsRequest - 6, // 13: xatu.Coordinator.CreateExecutionNodeRecordStatus:input_type -> xatu.CreateExecutionNodeRecordStatusRequest - 9, // 14: xatu.Coordinator.CoordinateExecutionNodeRecords:input_type -> xatu.CoordinateExecutionNodeRecordsRequest - 11, // 15: xatu.Coordinator.GetDiscoveryNodeRecord:input_type -> xatu.GetDiscoveryNodeRecordRequest - 17, // 16: xatu.Coordinator.GetCannonLocation:input_type -> xatu.GetCannonLocationRequest - 19, // 17: xatu.Coordinator.UpsertCannonLocation:input_type -> xatu.UpsertCannonLocationRequest - 2, // 18: xatu.Coordinator.CreateNodeRecords:output_type -> xatu.CreateNodeRecordsResponse - 4, // 19: xatu.Coordinator.ListStalledExecutionNodeRecords:output_type -> xatu.ListStalledExecutionNodeRecordsResponse - 7, // 20: xatu.Coordinator.CreateExecutionNodeRecordStatus:output_type -> xatu.CreateExecutionNodeRecordStatusResponse - 10, // 21: xatu.Coordinator.CoordinateExecutionNodeRecords:output_type -> xatu.CoordinateExecutionNodeRecordsResponse - 12, // 22: xatu.Coordinator.GetDiscoveryNodeRecord:output_type -> xatu.GetDiscoveryNodeRecordResponse - 18, // 23: xatu.Coordinator.GetCannonLocation:output_type -> xatu.GetCannonLocationResponse - 20, // 24: xatu.Coordinator.UpsertCannonLocation:output_type -> xatu.UpsertCannonLocationResponse - 18, // [18:25] is the sub-list for method output_type - 11, // [11:18] is the sub-list for method input_type - 11, // [11:11] is the sub-list for extension type_name - 11, // [11:11] is the sub-list for extension extendee - 0, // [0:11] is the sub-list for field type_name + 16, // 8: xatu.CannonLocation.eth_v2_beacon_block_attester_slashing:type_name -> xatu.CannonLocationEthV2BeaconBlockAttesterSlashing + 17, // 9: xatu.CannonLocation.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.CannonLocationEthV2BeaconBlockBlsToExecutionChange + 18, // 10: xatu.CannonLocation.eth_v2_beacon_block_execution_transaction:type_name -> xatu.CannonLocationEthV2BeaconBlockExecutionTransaction + 0, // 11: xatu.GetCannonLocationRequest.type:type_name -> xatu.CannonType + 19, // 12: xatu.GetCannonLocationResponse.location:type_name -> xatu.CannonLocation + 19, // 13: xatu.UpsertCannonLocationRequest.location:type_name -> xatu.CannonLocation + 1, // 14: xatu.Coordinator.CreateNodeRecords:input_type -> xatu.CreateNodeRecordsRequest + 3, // 15: xatu.Coordinator.ListStalledExecutionNodeRecords:input_type -> xatu.ListStalledExecutionNodeRecordsRequest + 6, // 16: xatu.Coordinator.CreateExecutionNodeRecordStatus:input_type -> xatu.CreateExecutionNodeRecordStatusRequest + 9, // 17: xatu.Coordinator.CoordinateExecutionNodeRecords:input_type -> xatu.CoordinateExecutionNodeRecordsRequest + 11, // 18: xatu.Coordinator.GetDiscoveryNodeRecord:input_type -> xatu.GetDiscoveryNodeRecordRequest + 20, // 19: xatu.Coordinator.GetCannonLocation:input_type -> xatu.GetCannonLocationRequest + 22, // 20: xatu.Coordinator.UpsertCannonLocation:input_type -> xatu.UpsertCannonLocationRequest + 2, // 21: xatu.Coordinator.CreateNodeRecords:output_type -> xatu.CreateNodeRecordsResponse + 4, // 22: xatu.Coordinator.ListStalledExecutionNodeRecords:output_type -> xatu.ListStalledExecutionNodeRecordsResponse + 7, // 23: xatu.Coordinator.CreateExecutionNodeRecordStatus:output_type -> xatu.CreateExecutionNodeRecordStatusResponse + 10, // 24: xatu.Coordinator.CoordinateExecutionNodeRecords:output_type -> xatu.CoordinateExecutionNodeRecordsResponse + 12, // 25: xatu.Coordinator.GetDiscoveryNodeRecord:output_type -> xatu.GetDiscoveryNodeRecordResponse + 21, // 26: xatu.Coordinator.GetCannonLocation:output_type -> xatu.GetCannonLocationResponse + 23, // 27: xatu.Coordinator.UpsertCannonLocation:output_type -> xatu.UpsertCannonLocationResponse + 21, // [21:28] is the sub-list for method output_type + 14, // [14:21] is the sub-list for method input_type + 14, // [14:14] is the sub-list for extension type_name + 14, // [14:14] is the sub-list for extension extendee + 0, // [0:14] is the sub-list for field type_name } func init() { file_pkg_proto_xatu_coordinator_proto_init() } @@ -1778,7 +2031,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CannonLocation); i { + switch v := v.(*CannonLocationEthV2BeaconBlockAttesterSlashing); i { case 0: return &v.state case 1: @@ -1790,7 +2043,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetCannonLocationRequest); i { + switch v := v.(*CannonLocationEthV2BeaconBlockBlsToExecutionChange); i { case 0: return &v.state case 1: @@ -1802,7 +2055,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetCannonLocationResponse); i { + switch v := v.(*CannonLocationEthV2BeaconBlockExecutionTransaction); i { case 0: return &v.state case 1: @@ -1814,7 +2067,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpsertCannonLocationRequest); i { + switch v := v.(*CannonLocation); i { case 0: return &v.state case 1: @@ -1826,7 +2079,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpsertCannonLocationResponse); i { + switch v := v.(*GetCannonLocationRequest); i { case 0: return &v.state case 1: @@ -1838,7 +2091,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutionNodeStatus_Capability); i { + switch v := v.(*GetCannonLocationResponse); i { case 0: return &v.state case 1: @@ -1850,6 +2103,42 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpsertCannonLocationRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_proto_xatu_coordinator_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpsertCannonLocationResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_proto_xatu_coordinator_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ExecutionNodeStatus_Capability); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_proto_xatu_coordinator_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecutionNodeStatus_ForkID); i { case 0: return &v.state @@ -1862,10 +2151,13 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } } - file_pkg_proto_xatu_coordinator_proto_msgTypes[15].OneofWrappers = []interface{}{ + file_pkg_proto_xatu_coordinator_proto_msgTypes[18].OneofWrappers = []interface{}{ (*CannonLocation_EthV2BeaconBlockVoluntaryExit)(nil), (*CannonLocation_EthV2BeaconBlockProposerSlashing)(nil), (*CannonLocation_EthV2BeaconBlockDeposit)(nil), + (*CannonLocation_EthV2BeaconBlockAttesterSlashing)(nil), + (*CannonLocation_EthV2BeaconBlockBlsToExecutionChange)(nil), + (*CannonLocation_EthV2BeaconBlockExecutionTransaction)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -1873,7 +2165,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_pkg_proto_xatu_coordinator_proto_rawDesc, NumEnums: 1, - NumMessages: 22, + NumMessages: 25, NumExtensions: 0, NumServices: 1, }, diff --git a/pkg/proto/xatu/coordinator.proto b/pkg/proto/xatu/coordinator.proto index 33f6102d..650df39a 100644 --- a/pkg/proto/xatu/coordinator.proto +++ b/pkg/proto/xatu/coordinator.proto @@ -92,6 +92,9 @@ enum CannonType { BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT = 0; BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING = 1; BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT = 2; + BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING = 3; + BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE = 4; + BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION = 5; } message CannonLocationEthV2BeaconBlockVoluntaryExit { @@ -106,6 +109,18 @@ message CannonLocationEthV2BeaconBlockDeposit { uint64 slot = 1; } +message CannonLocationEthV2BeaconBlockAttesterSlashing { + uint64 slot = 1; +} + +message CannonLocationEthV2BeaconBlockBlsToExecutionChange { + uint64 slot = 1; +} + +message CannonLocationEthV2BeaconBlockExecutionTransaction { + uint64 slot = 1; +} + message CannonLocation { string network_id = 1; CannonType type = 2; @@ -113,6 +128,9 @@ message CannonLocation { CannonLocationEthV2BeaconBlockVoluntaryExit eth_v2_beacon_block_voluntary_exit = 3 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT" ]; CannonLocationEthV2BeaconBlockProposerSlashing eth_v2_beacon_block_proposer_slashing = 4 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING" ]; CannonLocationEthV2BeaconBlockDeposit eth_v2_beacon_block_deposit = 5 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_DEPOSIT" ]; + CannonLocationEthV2BeaconBlockAttesterSlashing eth_v2_beacon_block_attester_slashing = 6 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING" ]; + CannonLocationEthV2BeaconBlockBlsToExecutionChange eth_v2_beacon_block_bls_to_execution_change = 7 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE" ]; + CannonLocationEthV2BeaconBlockExecutionTransaction eth_v2_beacon_block_execution_transaction = 8 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION" ]; } } diff --git a/pkg/proto/xatu/event_ingester.pb.go b/pkg/proto/xatu/event_ingester.pb.go index 146376aa..69047806 100644 --- a/pkg/proto/xatu/event_ingester.pb.go +++ b/pkg/proto/xatu/event_ingester.pb.go @@ -4647,6 +4647,10 @@ type ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData struct { Block *BlockIdentifier `protobuf:"bytes,1,opt,name=block,proto3" json:"block,omitempty"` // PositionInBlock is the position of the transaction in the block. PositionInBlock *wrapperspb.UInt64Value `protobuf:"bytes,2,opt,name=position_in_block,proto3" json:"position_in_block,omitempty"` + // Size is the transaction size in bytes. + Size string `protobuf:"bytes,3,opt,name=size,proto3" json:"size,omitempty"` + // CallDataSize is the call data size in bytes. + CallDataSize string `protobuf:"bytes,4,opt,name=call_data_size,proto3" json:"call_data_size,omitempty"` } func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) Reset() { @@ -4695,6 +4699,20 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) GetPosit return nil } +func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) GetSize() string { + if x != nil { + return x.Size + } + return "" +} + +func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) GetCallDataSize() string { + if x != nil { + return x.CallDataSize + } + return "" +} + type ClientMeta_AttestationDataSnapshot struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -5353,7 +5371,7 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6f, 0x74, 0x22, 0x8a, 0x5e, 0x0a, 0x0a, 0x43, 0x6c, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x6f, 0x6f, 0x74, 0x22, 0xc6, 0x5e, 0x0a, 0x0a, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, @@ -6055,7 +6073,7 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2b, 0x0a, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, - 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x1a, 0xad, 0x01, 0x0a, 0x32, 0x41, 0x64, 0x64, 0x69, + 0x52, 0x05, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x1a, 0xe9, 0x01, 0x0a, 0x32, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2b, @@ -6066,417 +6084,421 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x36, 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x11, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, - 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x1a, 0x8b, 0x02, 0x0a, 0x17, 0x41, 0x74, 0x74, 0x65, - 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6e, 0x61, 0x70, 0x73, - 0x68, 0x6f, 0x74, 0x12, 0x66, 0x0a, 0x1f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, - 0x5f, 0x61, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, - 0x69, 0x66, 0x66, 0x5f, 0x6d, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, - 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, - 0x49, 0x6e, 0x74, 0x36, 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x1f, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x13, 0x72, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x6d, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x63, + 0x61, 0x6c, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, + 0x69, 0x7a, 0x65, 0x1a, 0x8b, 0x02, 0x0a, 0x17, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, + 0x66, 0x0a, 0x1f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, + 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x5f, + 0x6d, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x36, - 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x13, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, - 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x1a, 0xdc, 0x02, 0x0a, 0x2b, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x6f, 0x72, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, - 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x4f, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x4f, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x52, - 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, - 0x6f, 0x63, 0x68, 0x56, 0x32, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x20, 0x0a, 0x04, - 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x44, - 0x0a, 0x08, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x28, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, - 0x74, 0x61, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, - 0x74, 0x61, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, - 0x73, 0x68, 0x6f, 0x74, 0x42, 0x10, 0x0a, 0x0e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x22, 0xd7, 0x05, 0x0a, 0x0a, 0x53, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, - 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x12, 0x2f, 0x0a, 0x06, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x63, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x12, 0x4e, 0x0a, 0x0f, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x61, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, + 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x1f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, + 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, + 0x64, 0x69, 0x66, 0x66, 0x5f, 0x6d, 0x73, 0x12, 0x4e, 0x0a, 0x13, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x36, 0x34, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x52, 0x13, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x73, 0x12, 0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, + 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x1a, 0xdc, 0x02, 0x0a, 0x2b, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, + 0x45, 0x74, 0x68, 0x56, 0x31, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x41, 0x74, + 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x44, 0x61, 0x74, + 0x61, 0x12, 0x4f, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x37, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, + 0x65, 0x74, 0x61, 0x2e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, + 0x68, 0x56, 0x31, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x12, 0x4f, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, + 0x74, 0x68, 0x56, 0x31, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x56, + 0x32, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x20, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, + 0x6f, 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x44, 0x0a, 0x08, 0x53, 0x6e, + 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x41, + 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x6e, + 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x42, 0x10, 0x0a, 0x0e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x44, 0x61, + 0x74, 0x61, 0x22, 0xd7, 0x05, 0x0a, 0x0a, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, + 0x61, 0x12, 0x2c, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, 0x65, + 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x12, + 0x2f, 0x0a, 0x06, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, + 0x61, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x12, 0x4e, 0x0a, 0x0f, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x41, 0x64, 0x64, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x0f, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, + 0x1a, 0x53, 0x0a, 0x05, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x4a, 0x0a, 0x12, 0x72, 0x65, 0x63, + 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x12, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x64, 0x61, 0x74, 0x65, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x1a, 0x81, 0x03, 0x0a, 0x06, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, + 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x70, + 0x12, 0x2d, 0x0a, 0x03, 0x67, 0x65, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, 0x2e, - 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x0f, 0x61, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, - 0x64, 0x61, 0x74, 0x61, 0x1a, 0x53, 0x0a, 0x05, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x4a, 0x0a, - 0x12, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, - 0x69, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x12, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, - 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x1a, 0x81, 0x03, 0x0a, 0x06, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x70, 0x12, 0x2d, 0x0a, 0x03, 0x67, 0x65, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, - 0x65, 0x74, 0x61, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x2e, 0x47, 0x65, 0x6f, 0x52, 0x03, - 0x67, 0x65, 0x6f, 0x1a, 0xb7, 0x02, 0x0a, 0x03, 0x47, 0x65, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x43, - 0x69, 0x74, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x69, 0x74, 0x79, 0x12, - 0x18, 0x0a, 0x07, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x21, 0x0a, 0x0b, 0x43, 0x6f, 0x75, - 0x6e, 0x74, 0x72, 0x79, 0x43, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0d, - 0x43, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x63, - 0x6f, 0x64, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x4c, 0x61, 0x74, 0x69, 0x74, 0x75, 0x64, 0x65, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x01, 0x52, 0x08, 0x6c, 0x61, 0x74, 0x69, 0x74, 0x75, 0x64, 0x65, 0x12, - 0x1c, 0x0a, 0x09, 0x4c, 0x6f, 0x6e, 0x67, 0x69, 0x74, 0x75, 0x64, 0x65, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x01, 0x52, 0x09, 0x6c, 0x6f, 0x6e, 0x67, 0x69, 0x74, 0x75, 0x64, 0x65, 0x12, 0x38, 0x0a, - 0x16, 0x41, 0x75, 0x74, 0x6f, 0x6e, 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x53, 0x79, 0x73, 0x74, 0x65, - 0x6d, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x18, 0x61, - 0x75, 0x74, 0x6f, 0x6e, 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x5f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, - 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x1c, 0x41, 0x75, 0x74, 0x6f, 0x6e, - 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4f, 0x72, 0x67, 0x61, 0x6e, - 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1e, 0x61, - 0x75, 0x74, 0x6f, 0x6e, 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x5f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, - 0x5f, 0x6f, 0x72, 0x67, 0x61, 0x6e, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x41, 0x0a, - 0x13, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x5a, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x28, 0x0a, 0x06, 0x63, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, - 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x06, 0x63, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x12, 0x28, 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, - 0x4d, 0x65, 0x74, 0x61, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x22, 0xec, 0x0b, 0x0a, - 0x05, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x2e, 0x4e, 0x61, 0x6d, 0x65, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x09, - 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x64, 0x61, 0x74, - 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0xf2, 0x0a, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x24, 0x0a, 0x20, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, - 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, - 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x22, 0x0a, 0x1e, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, - 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, - 0x53, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x01, 0x12, 0x28, 0x0a, 0x24, 0x42, 0x45, 0x41, - 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, - 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, - 0x47, 0x10, 0x02, 0x12, 0x31, 0x0a, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, - 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, - 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, 0x50, - 0x4f, 0x49, 0x4e, 0x54, 0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, - 0x54, 0x53, 0x5f, 0x48, 0x45, 0x41, 0x44, 0x10, 0x04, 0x12, 0x2b, 0x0a, 0x27, 0x42, 0x45, 0x41, - 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, - 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, - 0x45, 0x58, 0x49, 0x54, 0x10, 0x05, 0x12, 0x28, 0x0a, 0x24, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, - 0x54, 0x53, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, - 0x12, 0x33, 0x0a, 0x2f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, - 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, - 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x50, 0x52, - 0x4f, 0x4f, 0x46, 0x10, 0x07, 0x12, 0x17, 0x0a, 0x13, 0x4d, 0x45, 0x4d, 0x50, 0x4f, 0x4f, 0x4c, - 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x22, - 0x0a, 0x1e, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, - 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, - 0x10, 0x09, 0x12, 0x27, 0x0a, 0x23, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, - 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, - 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x10, 0x0a, 0x12, 0x2d, 0x0a, 0x29, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, - 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, - 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x10, 0x0b, 0x12, 0x26, 0x0a, 0x22, 0x42, 0x45, - 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, - 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x45, - 0x10, 0x0c, 0x12, 0x30, 0x0a, 0x2c, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, - 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, - 0x52, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x41, - 0x54, 0x41, 0x10, 0x0d, 0x12, 0x25, 0x0a, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, - 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, - 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x32, 0x10, 0x0e, 0x12, 0x2b, 0x0a, 0x27, 0x42, + 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x2e, 0x47, 0x65, 0x6f, 0x52, 0x03, 0x67, 0x65, 0x6f, 0x1a, + 0xb7, 0x02, 0x0a, 0x03, 0x47, 0x65, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x43, 0x69, 0x74, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x63, 0x69, 0x74, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x72, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x21, 0x0a, 0x0b, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x72, 0x79, + 0x43, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x72, 0x79, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x0a, 0x0d, 0x43, 0x6f, 0x6e, 0x74, + 0x69, 0x6e, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x12, + 0x1a, 0x0a, 0x08, 0x4c, 0x61, 0x74, 0x69, 0x74, 0x75, 0x64, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x01, 0x52, 0x08, 0x6c, 0x61, 0x74, 0x69, 0x74, 0x75, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x4c, + 0x6f, 0x6e, 0x67, 0x69, 0x74, 0x75, 0x64, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x01, 0x52, 0x09, + 0x6c, 0x6f, 0x6e, 0x67, 0x69, 0x74, 0x75, 0x64, 0x65, 0x12, 0x38, 0x0a, 0x16, 0x41, 0x75, 0x74, + 0x6f, 0x6e, 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x18, 0x61, 0x75, 0x74, 0x6f, 0x6e, + 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x5f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x6e, 0x75, 0x6d, + 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x1c, 0x41, 0x75, 0x74, 0x6f, 0x6e, 0x6f, 0x6d, 0x6f, 0x75, + 0x73, 0x53, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x4f, 0x72, 0x67, 0x61, 0x6e, 0x69, 0x7a, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1e, 0x61, 0x75, 0x74, 0x6f, 0x6e, + 0x6f, 0x6d, 0x6f, 0x75, 0x73, 0x5f, 0x73, 0x79, 0x73, 0x74, 0x65, 0x6d, 0x5f, 0x6f, 0x72, 0x67, + 0x61, 0x6e, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x41, 0x0a, 0x13, 0x41, 0x64, 0x64, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x5a, 0x0a, 0x04, + 0x4d, 0x65, 0x74, 0x61, 0x12, 0x28, 0x0a, 0x06, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x06, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x28, + 0x0a, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4d, 0x65, 0x74, 0x61, + 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x22, 0xec, 0x0b, 0x0a, 0x05, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x2e, 0x4e, 0x61, + 0x6d, 0x65, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x38, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x65, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, + 0x69, 0x64, 0x22, 0xf2, 0x0a, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, 0x0a, 0x20, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, - 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, - 0x4f, 0x52, 0x47, 0x5f, 0x56, 0x32, 0x10, 0x0f, 0x12, 0x34, 0x0a, 0x30, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, - 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x43, - 0x48, 0x45, 0x43, 0x4b, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x5f, 0x56, 0x32, 0x10, 0x10, 0x12, 0x24, - 0x0a, 0x20, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, - 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x48, 0x45, 0x41, 0x44, 0x5f, - 0x56, 0x32, 0x10, 0x11, 0x12, 0x2e, 0x0a, 0x2a, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, - 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, - 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x5f, - 0x56, 0x32, 0x10, 0x12, 0x12, 0x2b, 0x0a, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, - 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, - 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x56, 0x32, 0x10, - 0x13, 0x12, 0x36, 0x0a, 0x32, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, - 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x4f, - 0x4e, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x50, - 0x52, 0x4f, 0x4f, 0x46, 0x5f, 0x56, 0x32, 0x10, 0x14, 0x12, 0x1a, 0x0a, 0x16, 0x4d, 0x45, 0x4d, - 0x50, 0x4f, 0x4f, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x56, 0x32, 0x10, 0x15, 0x12, 0x25, 0x0a, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, - 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, - 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x32, 0x10, 0x16, 0x12, 0x2a, 0x0a, 0x26, + 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, + 0x00, 0x12, 0x22, 0x0a, 0x1e, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, + 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x42, 0x4c, + 0x4f, 0x43, 0x4b, 0x10, 0x01, 0x12, 0x28, 0x0a, 0x24, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, + 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x10, 0x02, 0x12, + 0x31, 0x0a, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, + 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x41, + 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, 0x50, 0x4f, 0x49, 0x4e, 0x54, + 0x10, 0x03, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x48, + 0x45, 0x41, 0x44, 0x10, 0x04, 0x12, 0x2b, 0x0a, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, + 0x53, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, + 0x10, 0x05, 0x12, 0x28, 0x0a, 0x24, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x41, + 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, 0x33, 0x0a, 0x2f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, - 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, - 0x49, 0x43, 0x45, 0x5f, 0x56, 0x32, 0x10, 0x17, 0x12, 0x30, 0x0a, 0x2c, 0x42, 0x45, 0x41, 0x43, + 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, 0x54, 0x52, 0x49, 0x42, + 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x4f, 0x46, 0x10, + 0x07, 0x12, 0x17, 0x0a, 0x13, 0x4d, 0x45, 0x4d, 0x50, 0x4f, 0x4f, 0x4c, 0x5f, 0x54, 0x52, 0x41, + 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x22, 0x0a, 0x1e, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x10, 0x09, 0x12, 0x27, + 0x0a, 0x23, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, + 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, + 0x48, 0x4f, 0x49, 0x43, 0x45, 0x10, 0x0a, 0x12, 0x2d, 0x0a, 0x29, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, + 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x5f, 0x52, + 0x45, 0x4f, 0x52, 0x47, 0x10, 0x0b, 0x12, 0x26, 0x0a, 0x22, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, + 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x42, 0x45, 0x41, 0x43, + 0x4f, 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x45, 0x10, 0x0c, 0x12, 0x30, + 0x0a, 0x2c, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, + 0x5f, 0x56, 0x31, 0x5f, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x41, 0x54, + 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x10, 0x0d, + 0x12, 0x25, 0x0a, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x42, 0x4c, 0x4f, + 0x43, 0x4b, 0x5f, 0x56, 0x32, 0x10, 0x0e, 0x12, 0x2b, 0x0a, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x5f, + 0x56, 0x32, 0x10, 0x0f, 0x12, 0x34, 0x0a, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, + 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, + 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, + 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x5f, 0x56, 0x32, 0x10, 0x10, 0x12, 0x24, 0x0a, 0x20, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, + 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x48, 0x45, 0x41, 0x44, 0x5f, 0x56, 0x32, 0x10, 0x11, + 0x12, 0x2e, 0x0a, 0x2a, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x56, 0x4f, 0x4c, + 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x5f, 0x56, 0x32, 0x10, 0x12, + 0x12, 0x2b, 0x0a, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x41, 0x54, 0x54, + 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x56, 0x32, 0x10, 0x13, 0x12, 0x36, 0x0a, + 0x32, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, + 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, 0x54, 0x52, 0x49, + 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x4f, 0x46, + 0x5f, 0x56, 0x32, 0x10, 0x14, 0x12, 0x1a, 0x0a, 0x16, 0x4d, 0x45, 0x4d, 0x50, 0x4f, 0x4f, 0x4c, + 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x56, 0x32, 0x10, + 0x15, 0x12, 0x25, 0x0a, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, + 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, + 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x32, 0x10, 0x16, 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x5f, - 0x52, 0x45, 0x4f, 0x52, 0x47, 0x5f, 0x56, 0x32, 0x10, 0x18, 0x12, 0x34, 0x0a, 0x30, 0x42, 0x45, - 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, - 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x41, 0x54, 0x54, - 0x45, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x19, - 0x12, 0x34, 0x0a, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, - 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, - 0x43, 0x4b, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, - 0x48, 0x49, 0x4e, 0x47, 0x10, 0x1a, 0x12, 0x31, 0x0a, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, + 0x56, 0x32, 0x10, 0x17, 0x12, 0x30, 0x0a, 0x2c, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, + 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, + 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4f, 0x52, + 0x47, 0x5f, 0x56, 0x32, 0x10, 0x18, 0x12, 0x34, 0x0a, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, - 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x10, 0x1b, 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x45, 0x41, - 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, - 0x53, 0x49, 0x54, 0x10, 0x1c, 0x12, 0x3a, 0x0a, 0x36, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x45, + 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x19, 0x12, 0x34, 0x0a, 0x30, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, + 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x50, + 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, + 0x10, 0x1a, 0x12, 0x31, 0x0a, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, + 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, + 0x58, 0x49, 0x54, 0x10, 0x1b, 0x12, 0x2a, 0x0a, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, - 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x42, 0x4c, 0x53, 0x5f, 0x54, 0x4f, 0x5f, 0x45, - 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, - 0x1d, 0x12, 0x38, 0x0a, 0x34, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, + 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x10, + 0x1c, 0x12, 0x3a, 0x0a, 0x36, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, - 0x4f, 0x43, 0x4b, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, - 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x1e, 0x22, 0xe8, 0x1a, 0x0a, 0x0e, - 0x44, 0x65, 0x63, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x21, - 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x12, 0x1e, 0x0a, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x04, 0x6d, 0x65, 0x74, - 0x61, 0x12, 0x67, 0x0a, 0x19, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, - 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, - 0x18, 0x01, 0x48, 0x00, 0x52, 0x24, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, - 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x41, - 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x5a, 0x0a, 0x13, 0x65, 0x74, - 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, - 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, - 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x1e, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, - 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, - 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x12, 0x6b, 0x0a, 0x19, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, - 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x72, 0x65, - 0x6f, 0x72, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x68, 0x61, - 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x24, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, - 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, - 0x4f, 0x52, 0x47, 0x12, 0x86, 0x01, 0x0a, 0x22, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, - 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, - 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x25, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x2d, 0x42, + 0x4f, 0x43, 0x4b, 0x5f, 0x42, 0x4c, 0x53, 0x5f, 0x54, 0x4f, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x10, 0x1d, 0x12, 0x38, 0x0a, + 0x34, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, + 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, + 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, + 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x1e, 0x22, 0xe8, 0x1a, 0x0a, 0x0e, 0x44, 0x65, 0x63, 0x6f, + 0x72, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x1e, 0x0a, + 0x04, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x04, 0x6d, 0x65, 0x74, 0x61, 0x12, 0x67, 0x0a, + 0x19, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x61, + 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x18, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, + 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, + 0x52, 0x24, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, + 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, + 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x5a, 0x0a, 0x13, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x02, 0x18, 0x01, + 0x48, 0x00, 0x52, 0x1e, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x42, 0x4c, 0x4f, + 0x43, 0x4b, 0x12, 0x6b, 0x0a, 0x19, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, + 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, + 0x6f, 0x72, 0x67, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x24, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x12, + 0x86, 0x01, 0x0a, 0x22, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x73, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x63, 0x68, 0x65, 0x63, + 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, + 0x69, 0x6e, 0x74, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, + 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x43, 0x48, + 0x45, 0x43, 0x4b, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x12, 0x57, 0x0a, 0x12, 0x65, 0x74, 0x68, 0x5f, + 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, + 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x42, 0x02, 0x18, 0x01, + 0x48, 0x00, 0x52, 0x1d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x48, 0x45, 0x41, + 0x44, 0x12, 0x74, 0x0a, 0x1c, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, + 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, + 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6e, + 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x27, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, + 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, + 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x12, 0x8b, 0x01, 0x0a, 0x24, 0x65, 0x74, 0x68, 0x5f, + 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, + 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x42, 0x02, + 0x18, 0x01, 0x48, 0x00, 0x52, 0x2f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, + 0x4f, 0x4e, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x44, 0x5f, + 0x50, 0x52, 0x4f, 0x4f, 0x46, 0x12, 0x36, 0x0a, 0x13, 0x6d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, + 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0a, 0x20, 0x01, + 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x13, 0x4d, 0x45, 0x4d, 0x50, 0x4f, 0x4f, + 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x5a, 0x0a, + 0x13, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, + 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, + 0x6f, 0x63, 0x6b, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x1e, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, + 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x12, 0x5e, 0x0a, 0x12, 0x65, 0x74, 0x68, + 0x5f, 0x76, 0x31, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x18, + 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, + 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x42, 0x02, + 0x18, 0x01, 0x48, 0x00, 0x52, 0x23, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, + 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x12, 0x6d, 0x0a, 0x18, 0x65, 0x74, 0x68, + 0x5f, 0x76, 0x31, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, + 0x72, 0x65, 0x6f, 0x72, 0x67, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, + 0x63, 0x65, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x29, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, - 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, - 0x44, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x12, 0x57, 0x0a, 0x12, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x68, 0x65, - 0x61, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, - 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, - 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x1d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, + 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, + 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x12, 0x5d, 0x0a, 0x17, 0x65, 0x74, 0x68, 0x5f, + 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, + 0x74, 0x65, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, + 0x65, 0x48, 0x00, 0x52, 0x22, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, + 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x43, 0x4f, + 0x4d, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x45, 0x12, 0x79, 0x0a, 0x21, 0x65, 0x74, 0x68, 0x5f, 0x76, + 0x31, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x74, 0x74, 0x65, + 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0f, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, + 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, + 0x56, 0x32, 0x48, 0x00, 0x52, 0x2c, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, + 0x52, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x41, + 0x54, 0x41, 0x12, 0x6b, 0x0a, 0x1c, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x76, 0x32, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x56, 0x32, 0x48, 0x00, 0x52, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, - 0x5f, 0x48, 0x45, 0x41, 0x44, 0x12, 0x74, 0x0a, 0x1c, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, - 0x5f, 0x65, 0x78, 0x69, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x56, - 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x42, 0x02, 0x18, 0x01, - 0x48, 0x00, 0x52, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, - 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x56, 0x4f, 0x4c, - 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x12, 0x8b, 0x01, 0x0a, 0x24, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, - 0x72, 0x6f, 0x6f, 0x66, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, - 0x6f, 0x66, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x2f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, + 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x56, 0x32, 0x12, + 0x5e, 0x0a, 0x16, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x32, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x48, 0x00, 0x52, 0x21, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, + 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x32, 0x12, + 0x6f, 0x0a, 0x1c, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x5f, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x5f, 0x76, 0x32, 0x18, + 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, + 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, + 0x6f, 0x72, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, 0x27, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, + 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x5f, 0x56, 0x32, + 0x12, 0x8a, 0x01, 0x0a, 0x25, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x63, 0x68, 0x65, + 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x32, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x27, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x56, 0x32, 0x48, 0x00, 0x52, 0x30, 0x42, 0x45, 0x41, + 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, + 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, + 0x43, 0x48, 0x45, 0x43, 0x4b, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x5f, 0x56, 0x32, 0x12, 0x5b, 0x0a, + 0x15, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x68, + 0x65, 0x61, 0x64, 0x5f, 0x76, 0x32, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x48, 0x65, 0x61, 0x64, 0x56, 0x32, 0x48, 0x00, 0x52, 0x20, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, - 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, - 0x41, 0x4e, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x4f, 0x46, 0x12, 0x36, 0x0a, 0x13, 0x6d, 0x65, 0x6d, - 0x70, 0x6f, 0x6f, 0x6c, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x13, 0x4d, 0x45, - 0x4d, 0x50, 0x4f, 0x4f, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, - 0x4e, 0x12, 0x5a, 0x0a, 0x13, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, - 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, - 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x1e, 0x42, + 0x54, 0x53, 0x5f, 0x48, 0x45, 0x41, 0x44, 0x5f, 0x56, 0x32, 0x12, 0x78, 0x0a, 0x1f, 0x65, 0x74, + 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x76, 0x6f, 0x6c, 0x75, + 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x76, 0x32, 0x18, 0x15, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, + 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, + 0x45, 0x78, 0x69, 0x74, 0x56, 0x32, 0x48, 0x00, 0x52, 0x2a, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, + 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, + 0x54, 0x53, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, 0x49, + 0x54, 0x5f, 0x56, 0x32, 0x12, 0x8f, 0x01, 0x0a, 0x27, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x5f, 0x76, 0x32, + 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, + 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x56, 0x32, + 0x48, 0x00, 0x52, 0x32, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, + 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x4f, 0x4e, + 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x50, 0x52, + 0x4f, 0x4f, 0x46, 0x5f, 0x56, 0x32, 0x12, 0x38, 0x0a, 0x16, 0x6d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, + 0x6c, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x32, + 0x18, 0x17, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x16, 0x4d, 0x45, 0x4d, 0x50, 0x4f, 0x4f, + 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x56, 0x32, + 0x12, 0x5e, 0x0a, 0x16, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, + 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x32, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x48, 0x00, 0x52, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, - 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x12, 0x5e, 0x0a, - 0x12, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, - 0x69, 0x63, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, - 0x63, 0x65, 0x42, 0x02, 0x18, 0x01, 0x48, 0x00, 0x52, 0x23, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, - 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x12, 0x6d, 0x0a, - 0x18, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, - 0x69, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, - 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x42, 0x02, 0x18, 0x01, 0x48, - 0x00, 0x52, 0x29, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, - 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, - 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x12, 0x5d, 0x0a, 0x17, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, - 0x69, 0x74, 0x74, 0x65, 0x65, 0x48, 0x00, 0x52, 0x22, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, - 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, - 0x4e, 0x5f, 0x43, 0x4f, 0x4d, 0x4d, 0x49, 0x54, 0x54, 0x45, 0x45, 0x12, 0x79, 0x0a, 0x21, 0x65, - 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f, - 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x61, 0x74, 0x61, - 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, - 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x44, 0x61, 0x74, 0x61, 0x56, 0x32, 0x48, 0x00, 0x52, 0x2c, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x56, 0x41, 0x4c, 0x49, - 0x44, 0x41, 0x54, 0x4f, 0x52, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, - 0x4e, 0x5f, 0x44, 0x41, 0x54, 0x41, 0x12, 0x6b, 0x0a, 0x1c, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, - 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x32, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x48, 0x00, 0x52, 0x27, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, - 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, - 0x5f, 0x56, 0x32, 0x12, 0x5e, 0x0a, 0x16, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x32, 0x18, 0x11, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, - 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x48, 0x00, - 0x52, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, - 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, - 0x5f, 0x56, 0x32, 0x12, 0x6f, 0x0a, 0x1c, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, - 0x5f, 0x76, 0x32, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x68, 0x61, - 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, 0x27, 0x42, 0x45, 0x41, - 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, - 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, - 0x47, 0x5f, 0x56, 0x32, 0x12, 0x8a, 0x01, 0x0a, 0x25, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, - 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x76, 0x32, 0x18, 0x13, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, - 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, - 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x56, 0x32, 0x48, 0x00, 0x52, - 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, - 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, - 0x5a, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, 0x50, 0x4f, 0x49, 0x4e, 0x54, 0x5f, 0x56, - 0x32, 0x12, 0x5b, 0x0a, 0x15, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x76, 0x32, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x18, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x64, 0x56, 0x32, 0x48, 0x00, 0x52, 0x20, 0x42, 0x45, - 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, - 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x48, 0x45, 0x41, 0x44, 0x5f, 0x56, 0x32, 0x12, 0x78, - 0x0a, 0x1f, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, - 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x5f, 0x76, - 0x32, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, - 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x56, 0x6f, 0x6c, 0x75, 0x6e, - 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x56, 0x32, 0x48, 0x00, 0x52, 0x2a, 0x42, 0x45, + 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x32, + 0x12, 0x62, 0x0a, 0x15, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, + 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x76, 0x32, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x6f, + 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x56, 0x32, 0x48, 0x00, 0x52, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, - 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, - 0x5f, 0x45, 0x58, 0x49, 0x54, 0x5f, 0x56, 0x32, 0x12, 0x8f, 0x01, 0x0a, 0x27, 0x65, 0x74, 0x68, - 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x63, 0x6f, 0x6e, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x6f, - 0x66, 0x5f, 0x76, 0x32, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, - 0x6f, 0x66, 0x56, 0x32, 0x48, 0x00, 0x52, 0x32, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, - 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x53, - 0x5f, 0x43, 0x4f, 0x4e, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x41, 0x4e, - 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x4f, 0x46, 0x5f, 0x56, 0x32, 0x12, 0x38, 0x0a, 0x16, 0x6d, 0x65, - 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x76, 0x32, 0x18, 0x17, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x16, 0x4d, 0x45, - 0x4d, 0x50, 0x4f, 0x4f, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, - 0x4e, 0x5f, 0x56, 0x32, 0x12, 0x5e, 0x0a, 0x16, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x32, 0x18, 0x18, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, - 0x76, 0x32, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x48, - 0x00, 0x52, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, - 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, - 0x4b, 0x5f, 0x56, 0x32, 0x12, 0x62, 0x0a, 0x15, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x66, - 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x76, 0x32, 0x18, 0x19, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, - 0x31, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x56, 0x32, 0x48, 0x00, - 0x52, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, - 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, - 0x48, 0x4f, 0x49, 0x43, 0x45, 0x5f, 0x56, 0x32, 0x12, 0x71, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, - 0x76, 0x31, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x72, - 0x65, 0x6f, 0x72, 0x67, 0x5f, 0x76, 0x32, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, - 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, 0x2c, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, - 0x5f, 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, - 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x5f, 0x56, 0x32, 0x12, 0x82, 0x01, 0x0a, 0x25, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, - 0x73, 0x68, 0x69, 0x6e, 0x67, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, - 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, 0x30, - 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, - 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x41, - 0x54, 0x54, 0x45, 0x53, 0x54, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, - 0x12, 0x82, 0x01, 0x0a, 0x25, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, - 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, - 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, - 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x56, - 0x32, 0x48, 0x00, 0x52, 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, + 0x44, 0x45, 0x42, 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, + 0x45, 0x5f, 0x56, 0x32, 0x12, 0x71, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x66, + 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, + 0x5f, 0x76, 0x32, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, + 0x52, 0x65, 0x6f, 0x72, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, 0x2c, 0x42, 0x45, 0x41, 0x43, 0x4f, + 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x44, 0x45, 0x42, + 0x55, 0x47, 0x5f, 0x46, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x48, 0x4f, 0x49, 0x43, 0x45, 0x5f, 0x52, + 0x45, 0x4f, 0x52, 0x47, 0x5f, 0x56, 0x32, 0x12, 0x82, 0x01, 0x0a, 0x25, 0x65, 0x74, 0x68, 0x5f, + 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, + 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x5f, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, + 0x67, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, + 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, + 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, 0x30, 0x42, 0x45, 0x41, 0x43, + 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, + 0x54, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x12, 0x82, 0x01, 0x0a, + 0x25, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x6c, + 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, + 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 0x56, 0x32, 0x48, 0x00, 0x52, + 0x30, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, + 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, + 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, 0x53, 0x48, 0x49, 0x4e, + 0x47, 0x12, 0x7f, 0x0a, 0x22, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, + 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, + 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, + 0x65, 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x56, + 0x32, 0x48, 0x00, 0x52, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, - 0x4f, 0x43, 0x4b, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x4c, 0x41, - 0x53, 0x48, 0x49, 0x4e, 0x47, 0x12, 0x7f, 0x0a, 0x22, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, - 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x76, 0x6f, 0x6c, - 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x5f, 0x65, 0x78, 0x69, 0x74, 0x18, 0x1d, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x22, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, - 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, - 0x78, 0x69, 0x74, 0x56, 0x32, 0x48, 0x00, 0x52, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, - 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, - 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, - 0x59, 0x5f, 0x45, 0x58, 0x49, 0x54, 0x12, 0x65, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, - 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, - 0x70, 0x6f, 0x73, 0x69, 0x74, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, - 0x74, 0x56, 0x32, 0x48, 0x00, 0x52, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, - 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, - 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x12, 0x98, 0x01, - 0x0a, 0x2b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x62, 0x6c, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x1f, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, - 0x32, 0x2e, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x4c, 0x53, 0x54, 0x6f, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x56, 0x32, 0x48, 0x00, - 0x52, 0x36, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, - 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, - 0x5f, 0x42, 0x4c, 0x53, 0x5f, 0x54, 0x4f, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, - 0x4e, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x47, 0x45, 0x12, 0x83, 0x01, 0x0a, 0x29, 0x65, 0x74, 0x68, - 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x34, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, - 0x4f, 0x4e, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x42, 0x06, - 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x32, 0x58, 0x0a, 0x0d, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, - 0x6e, 0x67, 0x65, 0x73, 0x74, 0x65, 0x72, 0x12, 0x47, 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x42, 0x2c, 0x5a, 0x2a, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x65, - 0x74, 0x68, 0x70, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x70, 0x73, 0x2f, 0x78, 0x61, 0x74, 0x75, 0x2f, - 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x78, 0x61, 0x74, 0x75, 0x62, 0x06, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x4f, 0x4c, 0x55, 0x4e, 0x54, 0x41, 0x52, 0x59, 0x5f, 0x45, 0x58, + 0x49, 0x54, 0x12, 0x65, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, + 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, + 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x56, 0x32, 0x48, + 0x00, 0x52, 0x26, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, + 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, + 0x4b, 0x5f, 0x44, 0x45, 0x50, 0x4f, 0x53, 0x49, 0x54, 0x12, 0x98, 0x01, 0x0a, 0x2b, 0x65, 0x74, + 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, + 0x6b, 0x5f, 0x62, 0x6c, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x32, 0x2e, 0x53, 0x69, + 0x67, 0x6e, 0x65, 0x64, 0x42, 0x4c, 0x53, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x56, 0x32, 0x48, 0x00, 0x52, 0x36, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x42, 0x4c, 0x53, + 0x5f, 0x54, 0x4f, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x43, 0x48, + 0x41, 0x4e, 0x47, 0x45, 0x12, 0x83, 0x01, 0x0a, 0x29, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, + 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x78, 0x65, + 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x34, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, + 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x32, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, + 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x45, 0x58, 0x45, 0x43, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x54, + 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x42, 0x06, 0x0a, 0x04, 0x64, 0x61, + 0x74, 0x61, 0x32, 0x58, 0x0a, 0x0d, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x67, 0x65, 0x73, + 0x74, 0x65, 0x72, 0x12, 0x47, 0x0a, 0x0c, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x2c, 0x5a, 0x2a, + 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x65, 0x74, 0x68, 0x70, 0x61, + 0x6e, 0x64, 0x61, 0x6f, 0x70, 0x73, 0x2f, 0x78, 0x61, 0x74, 0x75, 0x2f, 0x70, 0x6b, 0x67, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x78, 0x61, 0x74, 0x75, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x33, } var ( diff --git a/pkg/proto/xatu/event_ingester.proto b/pkg/proto/xatu/event_ingester.proto index 9779b4c4..5cd7ce78 100644 --- a/pkg/proto/xatu/event_ingester.proto +++ b/pkg/proto/xatu/event_ingester.proto @@ -551,6 +551,10 @@ message ClientMeta { BlockIdentifier block = 1; // PositionInBlock is the position of the transaction in the block. google.protobuf.UInt64Value position_in_block = 2 [ json_name = "position_in_block" ]; + // Size is the transaction size in bytes. + string size = 3; + // CallDataSize is the call data size in bytes. + string call_data_size = 4 [ json_name = "call_data_size" ]; } message AttestationDataSnapshot { diff --git a/pkg/server/persistence/cannon/location.go b/pkg/server/persistence/cannon/location.go index 4f537b89..948682ce 100644 --- a/pkg/server/persistence/cannon/location.go +++ b/pkg/server/persistence/cannon/location.go @@ -60,6 +60,41 @@ func (l *Location) Marshal(msg *xatu.CannonLocation) error { return err } + l.Value = string(b) + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING: + l.Type = "BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING" + + data := msg.GetEthV2BeaconBlockAttesterSlashing() + + b, err := protojson.Marshal(data) + if err != nil { + return err + } + + l.Value = string(b) + + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION: + l.Type = "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION" + + data := msg.GetEthV2BeaconBlockExecutionTransaction() + + b, err := protojson.Marshal(data) + if err != nil { + return err + } + + l.Value = string(b) + + case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE: + l.Type = "BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE" + + data := msg.GetEthV2BeaconBlockBlsToExecutionChange() + + b, err := protojson.Marshal(data) + if err != nil { + return err + } + l.Value = string(b) default: return fmt.Errorf("unknown type: %s", msg.Type) @@ -113,6 +148,47 @@ func (l *Location) Unmarshal() (*xatu.CannonLocation, error) { msg.Data = &xatu.CannonLocation_EthV2BeaconBlockDeposit{ EthV2BeaconBlockDeposit: data, } + case "BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING": + msg.Type = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ATTESTER_SLASHING + + data := &xatu.CannonLocationEthV2BeaconBlockAttesterSlashing{} + + err := protojson.Unmarshal([]byte(l.Value), data) + if err != nil { + return nil, err + } + + msg.Data = &xatu.CannonLocation_EthV2BeaconBlockAttesterSlashing{ + EthV2BeaconBlockAttesterSlashing: data, + } + + case "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION": + msg.Type = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION + + data := &xatu.CannonLocationEthV2BeaconBlockExecutionTransaction{} + + err := protojson.Unmarshal([]byte(l.Value), data) + if err != nil { + return nil, err + } + + msg.Data = &xatu.CannonLocation_EthV2BeaconBlockExecutionTransaction{ + EthV2BeaconBlockExecutionTransaction: data, + } + + case "BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE": + msg.Type = xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_BLS_TO_EXECUTION_CHANGE + + data := &xatu.CannonLocationEthV2BeaconBlockBlsToExecutionChange{} + + err := protojson.Unmarshal([]byte(l.Value), data) + if err != nil { + return nil, err + } + + msg.Data = &xatu.CannonLocation_EthV2BeaconBlockBlsToExecutionChange{ + EthV2BeaconBlockBlsToExecutionChange: data, + } default: return nil, fmt.Errorf("unknown type: %s", l.Type) }