diff --git a/deploy/local/docker-compose/vector-http-kafka.yaml b/deploy/local/docker-compose/vector-http-kafka.yaml index 71fdb64b..d450710c 100644 --- a/deploy/local/docker-compose/vector-http-kafka.yaml +++ b/deploy/local/docker-compose/vector-http-kafka.yaml @@ -66,6 +66,7 @@ transforms: libp2p_trace_gossipsub_beacon_block: .event.name == "LIBP2P_TRACE_GOSSIPSUB_BEACON_BLOCK" libp2p_trace_gossipsub_beacon_attestation: .event.name == "LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION" libp2p_trace_gossipsub_blob_sidecar: .event.name == "LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR" + beacon_api_eth_v1_beacon_validators: .event.name == "BEACON_API_ETH_V1_BEACON_VALIDATORS" sinks: metrics: type: prometheus_exporter @@ -873,3 +874,19 @@ sinks: enabled: true encoding: codec: json + beacon_api_eth_v1_beacon_validators_kafka: + type: kafka + buffer: + max_events: 500000 + batch: + timeout_secs: 0.5 + inputs: + - xatu_server_events_router.beacon_api_eth_v1_beacon_validators + bootstrap_servers: "${KAFKA_BROKERS}" + key_field: "event.id" + topic: beacon-api-eth-v1-beacon-validators + compression: snappy + healthcheck: + enabled: true + encoding: + codec: json \ No newline at end of file diff --git a/deploy/local/docker-compose/vector-kafka-clickhouse.yaml b/deploy/local/docker-compose/vector-kafka-clickhouse.yaml index b6f8cd88..55a95cf7 100644 --- a/deploy/local/docker-compose/vector-kafka-clickhouse.yaml +++ b/deploy/local/docker-compose/vector-kafka-clickhouse.yaml @@ -109,6 +109,18 @@ sources: - "^beacon-api-eth-v1-proposer-.+" librdkafka_options: message.max.bytes: "10485760" # 10MB + beacon_api_eth_v1_beacon_validators_kafka: + type: kafka + bootstrap_servers: "${KAFKA_BROKERS}" + auto_offset_reset: earliest + group_id: xatu-vector-kafka-clickhouse-beacon-api-eth-v1-beacon-validators + key_field: "event.id" + decoding: + codec: json + topics: + - "beacon-api-eth-v1-beacon-validators" + librdkafka_options: + message.max.bytes: "10485760" # 10MB transforms: xatu_server_events_meta: type: remap @@ -122,6 +134,7 @@ transforms: - beacon_api_eth_v1_beacon_blob_sidecar_kafka - beacon_p2p_events_kafka - beacon_api_eth_v1_proposer_kafka + - beacon_api_eth_v1_beacon_validators_kafka source: |- .meta_client_name = .meta.client.name .meta_client_id = .meta.client.id @@ -312,6 +325,7 @@ transforms: canonical_beacon_block_execution_transaction: .event.name == "BEACON_API_ETH_V2_BEACON_BLOCK_EXECUTION_TRANSACTION" canonical_beacon_block_proposer_slashing: .event.name == "BEACON_API_ETH_V2_BEACON_BLOCK_PROPOSER_SLASHING" canonical_beacon_block_voluntary_exit: .event.name == "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT" + canonical_beacon_validators: .event.name == "BEACON_API_ETH_V1_BEACON_VALIDATORS" canonical_beacon_block_withdrawal: .event.name == "BEACON_API_ETH_V2_BEACON_BLOCK_WITHDRAWAL" canonical_beacon_block: .event.name == "BEACON_API_ETH_V2_BEACON_BLOCK_V2" && .meta.client.additional_data.finalized_when_requested == true canonical_beacon_proposer_duty: .event.name == "BEACON_API_ETH_V1_PROPOSER_DUTY" && .meta.client.additional_data.state_id == "finalized" @@ -343,6 +357,7 @@ transforms: - xatu_server_events_router.beacon_p2p_attestation - xatu_server_events_router.blockprint_block_classification - xatu_server_events_router.canonical_beacon_blob_sidecar + - xatu_server_events_router.canonical_beacon_validators - xatu_server_events_router.canonical_beacon_block - xatu_server_events_router.canonical_beacon_block_attester_slashing - xatu_server_events_router.canonical_beacon_block_elaborated_attestation @@ -1455,6 +1470,56 @@ transforms: del(.event) del(.meta) del(.data) + canonical_beacon_validators_formatted: + type: remap + inputs: + - xatu_server_events_router.canonical_beacon_validators + source: |- + event_date_time, err = parse_timestamp(.event.date_time, format: "%+"); + if err == null { + .event_date_time = to_unix_timestamp(event_date_time, unit: "milliseconds") + } else { + .error = err + .error_description = "failed to parse event date time" + log(., level: "error", rate_limit_secs: 60) + } + .slot = .meta.client.additional_data.slot.number + slot_start_date_time, err = parse_timestamp(.meta.client.additional_data.slot.start_date_time, format: "%+"); + if err == null { + .slot_start_date_time = to_unix_timestamp(slot_start_date_time) + } else { + .error = err + .error_description = "failed to parse slot start date time" + log(., level: "error", rate_limit_secs: 60) + } + .epoch = .meta.client.additional_data.epoch.number + epoch_start_date_time, err = parse_timestamp(.meta.client.additional_data.epoch.start_date_time, format: "%+"); + if err == null { + .epoch_start_date_time = to_unix_timestamp(epoch_start_date_time) + } else { + .error = err + .error_description = "failed to parse epoch start date time" + log(., level: "error", rate_limit_secs: 60) + } + .block_root = .data.block_root + .block_parent_root = .data.block_parent_root + .versioned_hash = .meta.client.additional_data.versioned_hash + .kzg_commitment = .data.kzg_commitment + .kzg_proof = .data.kzg_proof + .proposer_index = .data.proposer_index + .blob_index = .data.index + .blob_size = .meta.client.additional_data.data_size + .blob_empty_size = .meta.client.additional_data.data_empty_size + key, err = .block_root + .versioned_hash + .blob_index + if err != null { + .error = err + .error_description = "failed to generate unique key" + } + .unique_key = seahash(key) + .updated_date_time = to_unix_timestamp(now()) + del(.event) + del(.meta) + del(.data) blockprint_block_classification_formatted: type: remap inputs: diff --git a/docker-compose.yml b/docker-compose.yml index 27928209..9d3c3ae0 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -265,6 +265,7 @@ services: "libp2p-trace-gossipsub-beacon-block" "libp2p-trace-gossipsub-beacon-attestation" "libp2p-trace-gossipsub-blob-sidecar" + "beacon-api-eth-v1-beacon-validators" ) for topic in "$${topics[@]}"; do echo "Creating topic: $$topic"; diff --git a/go.mod b/go.mod index c0d9dc83..e2d82c5a 100644 --- a/go.mod +++ b/go.mod @@ -13,7 +13,7 @@ require ( github.com/chuckpreslar/emission v0.0.0-20170206194824-a7ddd980baf9 github.com/creasty/defaults v1.7.0 github.com/ethereum/go-ethereum v1.13.15 - github.com/ethpandaops/beacon v0.35.0 + github.com/ethpandaops/beacon v0.37.0 github.com/ethpandaops/ethcore v0.0.0-20240422023000-2a5727b18756 github.com/ethpandaops/ethwallclock v0.3.0 github.com/go-co-op/gocron v1.27.1 diff --git a/go.sum b/go.sum index 5f71e1e4..9531a6ba 100644 --- a/go.sum +++ b/go.sum @@ -260,8 +260,8 @@ github.com/ethereum/c-kzg-4844 v0.4.0 h1:3MS1s4JtA868KpJxroZoepdV0ZKBp3u/O5HcZ7R github.com/ethereum/c-kzg-4844 v0.4.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= github.com/ethereum/go-ethereum v1.13.15 h1:U7sSGYGo4SPjP6iNIifNoyIAiNjrmQkz6EwQG+/EZWo= github.com/ethereum/go-ethereum v1.13.15/go.mod h1:TN8ZiHrdJwSe8Cb6x+p0hs5CxhJZPbqB7hHkaUXcmIU= -github.com/ethpandaops/beacon v0.35.0 h1:ZkHfxm41N0wkv503Xdb6rFxLuEnIonClUQWUPFHS5VU= -github.com/ethpandaops/beacon v0.35.0/go.mod h1:B+SLxj1gnDd/Ia7cl/uuhzo1wyVf2p2puL6lmzPdPro= +github.com/ethpandaops/beacon v0.37.0 h1:T+F0IEjkSrevAbGA4zsqvqjnm4IRp+JKLsd8DyAO8ZQ= +github.com/ethpandaops/beacon v0.37.0/go.mod h1:B+SLxj1gnDd/Ia7cl/uuhzo1wyVf2p2puL6lmzPdPro= github.com/ethpandaops/ethcore v0.0.0-20240422023000-2a5727b18756 h1:8JWjrRfP14m0oxOk03m11n/xgdY5ceyUf/ZxYdOs5gE= github.com/ethpandaops/ethcore v0.0.0-20240422023000-2a5727b18756/go.mod h1:ZvKqL6CKxiraefdXPHeJurV2pDD/f2HF2uklDVdrry8= github.com/ethpandaops/ethwallclock v0.3.0 h1:xF5fwtBf+bHFHZKBnwiPFEuelW3sMM7SD3ZNFq1lJY4= diff --git a/pkg/cannon/cannon.go b/pkg/cannon/cannon.go index 84bf812b..261bb0fd 100644 --- a/pkg/cannon/cannon.go +++ b/pkg/cannon/cannon.go @@ -548,6 +548,23 @@ func (c *Cannon) startBeaconBlockProcessor(ctx context.Context) error { c.beacon, clientMeta, ), + v1.NewBeaconValidatorsDeriver( + c.log, + &c.Config.Derivers.BeaconValidatorsConfig, + iterator.NewBackfillingCheckpoint( + c.log, + networkName, + networkID, + xatu.CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS, + c.coordinatorClient, + wallclock, + &backfillingCheckpointIteratorMetrics, + c.beacon, + finalizedCheckpoint, + ), + c.beacon, + clientMeta, + ), } c.eventDerivers = eventDerivers diff --git a/pkg/cannon/deriver/beacon/eth/v1/beacon_validators.go b/pkg/cannon/deriver/beacon/eth/v1/beacon_validators.go new file mode 100644 index 00000000..23a3b3b6 --- /dev/null +++ b/pkg/cannon/deriver/beacon/eth/v1/beacon_validators.go @@ -0,0 +1,353 @@ +package v1 + +import ( + "context" + "fmt" + "time" + + client "github.com/attestantio/go-eth2-client" + "github.com/attestantio/go-eth2-client/api" + apiv1 "github.com/attestantio/go-eth2-client/api/v1" + "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" + "github.com/ethpandaops/xatu/pkg/observability" + 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" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/codes" + "go.opentelemetry.io/otel/trace" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/types/known/timestamppb" + "google.golang.org/protobuf/types/known/wrapperspb" +) + +const ( + BeaconValidatorsDeriverName = xatu.CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS +) + +type BeaconValidatorsDeriverConfig struct { + Enabled bool `yaml:"enabled" default:"true"` +} + +type BeaconValidatorsDeriver struct { + log logrus.FieldLogger + cfg *BeaconValidatorsDeriverConfig + iterator *iterator.BackfillingCheckpoint + onEventsCallbacks []func(ctx context.Context, events []*xatu.DecoratedEvent) error + beacon *ethereum.BeaconNode + clientMeta *xatu.ClientMeta +} + +func NewBeaconValidatorsDeriver(log logrus.FieldLogger, config *BeaconValidatorsDeriverConfig, iter *iterator.BackfillingCheckpoint, beacon *ethereum.BeaconNode, clientMeta *xatu.ClientMeta) *BeaconValidatorsDeriver { + return &BeaconValidatorsDeriver{ + log: log.WithField("module", "cannon/event/beacon/eth/v1/validators"), + cfg: config, + iterator: iter, + beacon: beacon, + clientMeta: clientMeta, + } +} + +func (b *BeaconValidatorsDeriver) CannonType() xatu.CannonType { + return BeaconValidatorsDeriverName +} + +func (b *BeaconValidatorsDeriver) ActivationFork() string { + return ethereum.ForkNamePhase0 +} + +func (b *BeaconValidatorsDeriver) Name() string { + return BeaconValidatorsDeriverName.String() +} + +func (b *BeaconValidatorsDeriver) OnEventsDerived(ctx context.Context, fn func(ctx context.Context, events []*xatu.DecoratedEvent) error) { + b.onEventsCallbacks = append(b.onEventsCallbacks, fn) +} + +func (b *BeaconValidatorsDeriver) Start(ctx context.Context) error { + if !b.cfg.Enabled { + b.log.Info("Validator states deriver disabled") + + return nil + } + + b.log.Info("Validator states deriver enabled") + + // Start our main loop + b.run(ctx) + + return nil +} + +func (b *BeaconValidatorsDeriver) Stop(ctx context.Context) error { + return nil +} + +func (b *BeaconValidatorsDeriver) run(rctx context.Context) { + bo := backoff.NewExponentialBackOff() + bo.MaxInterval = 3 * time.Minute + + tracer := observability.Tracer() + + for { + select { + case <-rctx.Done(): + return + default: + operation := func() error { + ctx, span := tracer.Start(rctx, fmt.Sprintf("Derive %s", b.Name()), + trace.WithAttributes( + attribute.String("network", string(b.beacon.Metadata().Network.Name))), + ) + defer span.End() + + time.Sleep(100 * time.Millisecond) + + if err := b.beacon.Synced(ctx); err != nil { + span.SetStatus(codes.Error, err.Error()) + + return err + } + + // Get the next location. + location, _, err := b.iterator.Next(ctx) + if err != nil { + span.SetStatus(codes.Error, err.Error()) + + return err + } + + finality, err := b.beacon.Node().Finality() + if err != nil { + span.SetStatus(codes.Error, err.Error()) + + return errors.Wrap(err, "failed to get finality") + } + + if finality == nil { + span.SetStatus(codes.Error, "finality is nil") + + return errors.New("finality is nil") + } + + checkpoint := finality.Finalized + + processingFinalizedEpoch := false + epochToProcess := location.GetEthV1BeaconValidators().GetBackfillingCheckpointMarker().GetBackfillEpoch() - 1 + + if checkpoint.Epoch > phase0.Epoch(location.GetEthV1BeaconValidators().BackfillingCheckpointMarker.FinalizedEpoch) { + processingFinalizedEpoch = true + epochToProcess = int64(location.GetEthV1BeaconValidators().BackfillingCheckpointMarker.FinalizedEpoch) + 1 + } + + if !processingFinalizedEpoch && epochToProcess == -1 { + return nil + } + + events, err := b.processEpoch(ctx, phase0.Epoch(epochToProcess)) + if err != nil { + b.log.WithError(err).WithField("epoch", epochToProcess).Error("Failed to process epoch") + + span.SetStatus(codes.Error, err.Error()) + + return err + } + + for _, fn := range b.onEventsCallbacks { + if err := fn(ctx, events); err != nil { + span.SetStatus(codes.Error, err.Error()) + + return errors.Wrap(err, "failed to send events") + } + } + + if processingFinalizedEpoch { + location.Data = &xatu.CannonLocation_EthV1BeaconValidators{ + EthV1BeaconValidators: &xatu.CannonLocationEthV1BeaconValidators{ + BackfillingCheckpointMarker: &xatu.BackfillingCheckpointMarker{ + BackfillEpoch: location.GetEthV1BeaconValidators().GetBackfillingCheckpointMarker().GetBackfillEpoch(), + FinalizedEpoch: uint64(epochToProcess), + }, + }, + } + } else { + location.Data = &xatu.CannonLocation_EthV1BeaconValidators{ + EthV1BeaconValidators: &xatu.CannonLocationEthV1BeaconValidators{ + BackfillingCheckpointMarker: &xatu.BackfillingCheckpointMarker{ + BackfillEpoch: epochToProcess, + FinalizedEpoch: location.GetEthV1BeaconValidators().GetBackfillingCheckpointMarker().GetFinalizedEpoch(), + }, + }, + } + } + + span.SetAttributes(attribute.Int64("epoch", epochToProcess)) + span.SetAttributes(attribute.Bool("is_backfill", !processingFinalizedEpoch)) + + if err := b.iterator.UpdateLocation(ctx, location); err != nil { + span.SetStatus(codes.Error, err.Error()) + + 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 *BeaconValidatorsDeriver) processEpoch(ctx context.Context, epoch phase0.Epoch) ([]*xatu.DecoratedEvent, error) { + ctx, span := observability.Tracer().Start(ctx, + "BeaconValidatorsDeriver.processEpoch", + trace.WithAttributes(attribute.Int64("epoch", int64(epoch))), + ) + defer span.End() + + validatorStates, err := b.getValidatorsClient(ctx) + if err != nil { + return nil, errors.Wrap(err, "failed to fetch validator states") + } + + spec, err := b.beacon.Node().Spec() + if err != nil { + return nil, errors.Wrap(err, "failed to fetch spec") + } + + boundarySlot := phase0.Slot(uint64(epoch) * uint64(spec.SlotsPerEpoch)) + + validatorsResponse, err := validatorStates.Validators(ctx, &api.ValidatorsOpts{ + State: xatuethv1.SlotAsString(boundarySlot), + Common: api.CommonOpts{ + Timeout: 6 * time.Minute, // If we can't fetch 1 epoch of validators within 1 epoch we will never catch up. + }, + }) + if err != nil { + return nil, errors.Wrap(err, "failed to fetch validator states") + } + + allEvents := []*xatu.DecoratedEvent{} + validatorsMap := validatorsResponse.Data + + // Chunk the validators per 10000 + chunkSize := 10000 + + var validatorChunks [][]*apiv1.Validator + + currentChunk := []*apiv1.Validator{} + + for _, validator := range validatorsMap { + if len(currentChunk) == chunkSize { + validatorChunks = append(validatorChunks, currentChunk) + currentChunk = []*apiv1.Validator{} + } + + currentChunk = append(currentChunk, validator) + } + + if len(currentChunk) > 0 { + validatorChunks = append(validatorChunks, currentChunk) + } + + for chunkNum, chunk := range validatorChunks { + event, err := b.createEventFromValidators(ctx, chunk, epoch) + if err != nil { + b.log. + WithError(err). + WithField("chunk_size", len(chunk)). + WithField("chunk_number", chunkNum). + WithField("epoch", epoch). + Error("Failed to create event from validator state") + + return nil, err + } + + allEvents = append(allEvents, event) + } + + return allEvents, nil +} + +func (b *BeaconValidatorsDeriver) getValidatorsClient(ctx context.Context) (client.ValidatorsProvider, error) { + if provider, isProvider := b.beacon.Node().Service().(client.ValidatorsProvider); isProvider { + return provider, nil + } + + return nil, errors.New("validator states client not found") +} + +func (b *BeaconValidatorsDeriver) createEventFromValidators(ctx context.Context, validators []*apiv1.Validator, epoch phase0.Epoch) (*xatu.DecoratedEvent, error) { + metadata, ok := proto.Clone(b.clientMeta).(*xatu.ClientMeta) + if !ok { + return nil, errors.New("failed to clone client metadata") + } + + data := xatu.Validators{} + for _, validator := range validators { + data.Validators = append(data.Validators, &xatuethv1.Validator{ + Index: wrapperspb.UInt64(uint64(validator.Index)), + Balance: wrapperspb.UInt64(uint64(validator.Balance)), + Status: wrapperspb.String(validator.Status.String()), + Data: &xatuethv1.ValidatorData{ + Pubkey: wrapperspb.String(validator.Validator.PublicKey.String()), + WithdrawalCredentials: wrapperspb.String(fmt.Sprintf("%#x", validator.Validator.WithdrawalCredentials)), + EffectiveBalance: wrapperspb.UInt64(uint64(validator.Validator.EffectiveBalance)), + Slashed: wrapperspb.Bool(validator.Validator.Slashed), + ActivationEpoch: wrapperspb.UInt64(uint64(validator.Validator.ActivationEpoch)), + ActivationEligibilityEpoch: wrapperspb.UInt64(uint64(validator.Validator.ActivationEligibilityEpoch)), + ExitEpoch: wrapperspb.UInt64(uint64(validator.Validator.ExitEpoch)), + WithdrawableEpoch: wrapperspb.UInt64(uint64(validator.Validator.WithdrawableEpoch)), + }, + }) + } + + decoratedEvent := &xatu.DecoratedEvent{ + Event: &xatu.Event{ + Name: xatu.Event_BEACON_API_ETH_V1_BEACON_VALIDATORS, + DateTime: timestamppb.New(time.Now()), + Id: uuid.New().String(), + }, + Meta: &xatu.Meta{ + Client: metadata, + }, + Data: &xatu.DecoratedEvent_EthV1Validators{ + EthV1Validators: &data, + }, + } + + additionalData, err := b.getAdditionalData(ctx, epoch) + if err != nil { + b.log.WithError(err).Error("Failed to get extra validator state data") + + return nil, err + } else { + decoratedEvent.Meta.Client.AdditionalData = &xatu.ClientMeta_EthV1Validators{ + EthV1Validators: additionalData, + } + } + + return decoratedEvent, nil +} + +func (b *BeaconValidatorsDeriver) getAdditionalData(_ context.Context, epoch phase0.Epoch) (*xatu.ClientMeta_AdditionalEthV1ValidatorsData, error) { + epochInfo := b.beacon.Metadata().Wallclock().Epochs().FromNumber(uint64(epoch)) + + return &xatu.ClientMeta_AdditionalEthV1ValidatorsData{ + Epoch: &xatu.EpochV2{ + Number: &wrapperspb.UInt64Value{Value: uint64(epoch)}, + StartDateTime: timestamppb.New(epochInfo.TimeWindow().Start()), + }, + }, nil +} diff --git a/pkg/cannon/deriver/config.go b/pkg/cannon/deriver/config.go index fb7e9c1a..dba072e3 100644 --- a/pkg/cannon/deriver/config.go +++ b/pkg/cannon/deriver/config.go @@ -20,6 +20,7 @@ type Config struct { BeaconBlobSidecarConfig v1.BeaconBlobDeriverConfig `yaml:"beaconBlobSidecar"` ProposerDutyConfig v1.ProposerDutyDeriverConfig `yaml:"proposerDuty"` ElaboratedAttestationConfig v2.ElaboratedAttestationDeriverConfig `yaml:"elaboratedAttestation"` + BeaconValidatorsConfig v1.BeaconValidatorsDeriverConfig `yaml:"beaconValidators"` } func (c *Config) Validate() error { diff --git a/pkg/cannon/deriver/event_deriver.go b/pkg/cannon/deriver/event_deriver.go index a4853d33..5a4de5fb 100644 --- a/pkg/cannon/deriver/event_deriver.go +++ b/pkg/cannon/deriver/event_deriver.go @@ -33,3 +33,4 @@ var _ EventDeriver = &blockprint.BlockClassificationDeriver{} var _ EventDeriver = &v2.ElaboratedAttestationDeriver{} var _ EventDeriver = &v1.ProposerDutyDeriver{} var _ EventDeriver = &v1.BeaconBlobDeriver{} +var _ EventDeriver = &v1.BeaconValidatorsDeriver{} diff --git a/pkg/cannon/iterator/backfilling_checkpoint_iterator.go b/pkg/cannon/iterator/backfilling_checkpoint_iterator.go index c45c9f44..6a8fd3f7 100644 --- a/pkg/cannon/iterator/backfilling_checkpoint_iterator.go +++ b/pkg/cannon/iterator/backfilling_checkpoint_iterator.go @@ -165,6 +165,8 @@ func (c *BackfillingCheckpoint) getMarker(location *xatu.CannonLocation) (*xatu. return location.GetEthV1BeaconProposerDuty().GetBackfillingCheckpointMarker(), nil case xatu.CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION: return location.GetEthV2BeaconBlockElaboratedAttestation().GetBackfillingCheckpointMarker(), nil + case xatu.CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS: + return location.GetEthV1BeaconValidators().GetBackfillingCheckpointMarker(), nil default: return nil, errors.Errorf("unknown cannon type %s", location.Type) } @@ -195,6 +197,15 @@ func (c *BackfillingCheckpoint) createLocationFromEpochNumber(finalized, backfil }, }, } + case xatu.CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS: + location.Data = &xatu.CannonLocation_EthV1BeaconValidators{ + EthV1BeaconValidators: &xatu.CannonLocationEthV1BeaconValidators{ + BackfillingCheckpointMarker: &xatu.BackfillingCheckpointMarker{ + FinalizedEpoch: uint64(finalized), + BackfillEpoch: int64(backfill), + }, + }, + } default: return location, errors.Errorf("unknown cannon type %s", location.Type) } diff --git a/pkg/proto/blockprint/block_classification.pb.go b/pkg/proto/blockprint/block_classification.pb.go index 59877567..7dacf5a3 100644 --- a/pkg/proto/blockprint/block_classification.pb.go +++ b/pkg/proto/blockprint/block_classification.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/blockprint/block_classification.proto package blockprint diff --git a/pkg/proto/eth/v1/attestation.pb.go b/pkg/proto/eth/v1/attestation.pb.go index a48218b7..e88179b6 100644 --- a/pkg/proto/eth/v1/attestation.pb.go +++ b/pkg/proto/eth/v1/attestation.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/attestation.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/beacon_block.pb.go b/pkg/proto/eth/v1/beacon_block.pb.go index 7f1e0cbd..6bd7c4cc 100644 --- a/pkg/proto/eth/v1/beacon_block.pb.go +++ b/pkg/proto/eth/v1/beacon_block.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/beacon_block.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/blob.pb.go b/pkg/proto/eth/v1/blob.pb.go index 708e16f0..4eacdc89 100644 --- a/pkg/proto/eth/v1/blob.pb.go +++ b/pkg/proto/eth/v1/blob.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/blob.proto package v1 diff --git a/pkg/proto/eth/v1/checkpoint.pb.go b/pkg/proto/eth/v1/checkpoint.pb.go index de681c73..8eb4e410 100644 --- a/pkg/proto/eth/v1/checkpoint.pb.go +++ b/pkg/proto/eth/v1/checkpoint.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/checkpoint.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/committee.pb.go b/pkg/proto/eth/v1/committee.pb.go index 7a07a4f1..8108585b 100644 --- a/pkg/proto/eth/v1/committee.pb.go +++ b/pkg/proto/eth/v1/committee.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/committee.proto // Note: largely inspired by https://github.com/prysmaticlabs/prysm/tree/develop/proto/eth/v1 diff --git a/pkg/proto/eth/v1/duties.pb.go b/pkg/proto/eth/v1/duties.pb.go index a5502f64..31987514 100644 --- a/pkg/proto/eth/v1/duties.pb.go +++ b/pkg/proto/eth/v1/duties.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/duties.proto package v1 diff --git a/pkg/proto/eth/v1/events.pb.go b/pkg/proto/eth/v1/events.pb.go index 533039ec..45e146e5 100644 --- a/pkg/proto/eth/v1/events.pb.go +++ b/pkg/proto/eth/v1/events.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/events.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/execution_engine.pb.go b/pkg/proto/eth/v1/execution_engine.pb.go index d86ea3a6..b0ec27c7 100644 --- a/pkg/proto/eth/v1/execution_engine.pb.go +++ b/pkg/proto/eth/v1/execution_engine.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/execution_engine.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/fork_choice.pb.go b/pkg/proto/eth/v1/fork_choice.pb.go index 6aa08831..73108ac4 100644 --- a/pkg/proto/eth/v1/fork_choice.pb.go +++ b/pkg/proto/eth/v1/fork_choice.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/fork_choice.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/sync_committee.pb.go b/pkg/proto/eth/v1/sync_committee.pb.go index fef4b82c..43123452 100644 --- a/pkg/proto/eth/v1/sync_committee.pb.go +++ b/pkg/proto/eth/v1/sync_committee.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v1/sync_committee.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v1/validator.pb.go b/pkg/proto/eth/v1/validator.pb.go new file mode 100644 index 00000000..dd1ff97f --- /dev/null +++ b/pkg/proto/eth/v1/validator.pb.go @@ -0,0 +1,352 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.31.0 +// protoc v4.24.2 +// source: pkg/proto/eth/v1/validator.proto + +package v1 + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + wrapperspb "google.golang.org/protobuf/types/known/wrapperspb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type ValidatorData struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Pubkey *wrapperspb.StringValue `protobuf:"bytes,1,opt,name=pubkey,proto3" json:"pubkey,omitempty"` + WithdrawalCredentials *wrapperspb.StringValue `protobuf:"bytes,2,opt,name=withdrawal_credentials,proto3" json:"withdrawal_credentials,omitempty"` + EffectiveBalance *wrapperspb.UInt64Value `protobuf:"bytes,3,opt,name=effective_balance,proto3" json:"effective_balance,omitempty"` + Slashed *wrapperspb.BoolValue `protobuf:"bytes,4,opt,name=slashed,proto3" json:"slashed,omitempty"` + ActivationEligibilityEpoch *wrapperspb.UInt64Value `protobuf:"bytes,5,opt,name=activation_eligibility_epoch,proto3" json:"activation_eligibility_epoch,omitempty"` + ActivationEpoch *wrapperspb.UInt64Value `protobuf:"bytes,6,opt,name=activation_epoch,proto3" json:"activation_epoch,omitempty"` + ExitEpoch *wrapperspb.UInt64Value `protobuf:"bytes,7,opt,name=exit_epoch,proto3" json:"exit_epoch,omitempty"` + WithdrawableEpoch *wrapperspb.UInt64Value `protobuf:"bytes,8,opt,name=withdrawable_epoch,proto3" json:"withdrawable_epoch,omitempty"` +} + +func (x *ValidatorData) Reset() { + *x = ValidatorData{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_eth_v1_validator_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ValidatorData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ValidatorData) ProtoMessage() {} + +func (x *ValidatorData) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_eth_v1_validator_proto_msgTypes[0] + 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 ValidatorData.ProtoReflect.Descriptor instead. +func (*ValidatorData) Descriptor() ([]byte, []int) { + return file_pkg_proto_eth_v1_validator_proto_rawDescGZIP(), []int{0} +} + +func (x *ValidatorData) GetPubkey() *wrapperspb.StringValue { + if x != nil { + return x.Pubkey + } + return nil +} + +func (x *ValidatorData) GetWithdrawalCredentials() *wrapperspb.StringValue { + if x != nil { + return x.WithdrawalCredentials + } + return nil +} + +func (x *ValidatorData) GetEffectiveBalance() *wrapperspb.UInt64Value { + if x != nil { + return x.EffectiveBalance + } + return nil +} + +func (x *ValidatorData) GetSlashed() *wrapperspb.BoolValue { + if x != nil { + return x.Slashed + } + return nil +} + +func (x *ValidatorData) GetActivationEligibilityEpoch() *wrapperspb.UInt64Value { + if x != nil { + return x.ActivationEligibilityEpoch + } + return nil +} + +func (x *ValidatorData) GetActivationEpoch() *wrapperspb.UInt64Value { + if x != nil { + return x.ActivationEpoch + } + return nil +} + +func (x *ValidatorData) GetExitEpoch() *wrapperspb.UInt64Value { + if x != nil { + return x.ExitEpoch + } + return nil +} + +func (x *ValidatorData) GetWithdrawableEpoch() *wrapperspb.UInt64Value { + if x != nil { + return x.WithdrawableEpoch + } + return nil +} + +type Validator struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Data *ValidatorData `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` + Index *wrapperspb.UInt64Value `protobuf:"bytes,2,opt,name=index,proto3" json:"index,omitempty"` + Balance *wrapperspb.UInt64Value `protobuf:"bytes,3,opt,name=balance,proto3" json:"balance,omitempty"` + Status *wrapperspb.StringValue `protobuf:"bytes,4,opt,name=status,proto3" json:"status,omitempty"` +} + +func (x *Validator) Reset() { + *x = Validator{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_eth_v1_validator_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Validator) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Validator) ProtoMessage() {} + +func (x *Validator) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_eth_v1_validator_proto_msgTypes[1] + 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 Validator.ProtoReflect.Descriptor instead. +func (*Validator) Descriptor() ([]byte, []int) { + return file_pkg_proto_eth_v1_validator_proto_rawDescGZIP(), []int{1} +} + +func (x *Validator) GetData() *ValidatorData { + if x != nil { + return x.Data + } + return nil +} + +func (x *Validator) GetIndex() *wrapperspb.UInt64Value { + if x != nil { + return x.Index + } + return nil +} + +func (x *Validator) GetBalance() *wrapperspb.UInt64Value { + if x != nil { + return x.Balance + } + return nil +} + +func (x *Validator) GetStatus() *wrapperspb.StringValue { + if x != nil { + return x.Status + } + return nil +} + +var File_pkg_proto_eth_v1_validator_proto protoreflect.FileDescriptor + +var file_pkg_proto_eth_v1_validator_proto_rawDesc = []byte{ + 0x0a, 0x20, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x68, 0x2f, + 0x76, 0x31, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x12, 0x0b, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x1a, + 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2f, 0x77, 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, + 0xd5, 0x04, 0x0a, 0x0d, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x44, 0x61, 0x74, + 0x61, 0x12, 0x34, 0x0a, 0x06, 0x70, 0x75, 0x62, 0x6b, 0x65, 0x79, 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, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x06, 0x70, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x12, 0x54, 0x0a, 0x16, 0x77, 0x69, 0x74, 0x68, 0x64, + 0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, + 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, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, + 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x16, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, + 0x6c, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x12, 0x4a, 0x0a, + 0x11, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, 0x65, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x18, 0x03, 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, 0x65, 0x66, 0x66, 0x65, 0x63, 0x74, 0x69, 0x76, + 0x65, 0x5f, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x07, 0x73, 0x6c, 0x61, + 0x73, 0x68, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x42, 0x6f, 0x6f, + 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x07, 0x73, 0x6c, 0x61, 0x73, 0x68, 0x65, 0x64, 0x12, + 0x60, 0x0a, 0x1c, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6c, + 0x69, 0x67, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, + 0x05, 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, 0x1c, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x65, 0x6c, 0x69, 0x67, 0x69, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x79, 0x5f, 0x65, 0x70, 0x6f, 0x63, + 0x68, 0x12, 0x48, 0x0a, 0x10, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x06, 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, 0x10, 0x61, 0x63, 0x74, 0x69, 0x76, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x3c, 0x0a, 0x0a, 0x65, + 0x78, 0x69, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x07, 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, 0x0a, 0x65, + 0x78, 0x69, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x4c, 0x0a, 0x12, 0x77, 0x69, 0x74, + 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, + 0x08, 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, 0x12, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x62, 0x6c, + 0x65, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x22, 0xdd, 0x01, 0x0a, 0x09, 0x56, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x2e, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, + 0x31, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x44, 0x61, 0x74, 0x61, 0x52, + 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 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, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x36, 0x0a, 0x07, 0x62, 0x61, 0x6c, + 0x61, 0x6e, 0x63, 0x65, 0x18, 0x03, 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, 0x07, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, + 0x65, 0x12, 0x34, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x42, 0x2e, 0x5a, 0x2c, 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, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_pkg_proto_eth_v1_validator_proto_rawDescOnce sync.Once + file_pkg_proto_eth_v1_validator_proto_rawDescData = file_pkg_proto_eth_v1_validator_proto_rawDesc +) + +func file_pkg_proto_eth_v1_validator_proto_rawDescGZIP() []byte { + file_pkg_proto_eth_v1_validator_proto_rawDescOnce.Do(func() { + file_pkg_proto_eth_v1_validator_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_proto_eth_v1_validator_proto_rawDescData) + }) + return file_pkg_proto_eth_v1_validator_proto_rawDescData +} + +var file_pkg_proto_eth_v1_validator_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_pkg_proto_eth_v1_validator_proto_goTypes = []interface{}{ + (*ValidatorData)(nil), // 0: xatu.eth.v1.ValidatorData + (*Validator)(nil), // 1: xatu.eth.v1.Validator + (*wrapperspb.StringValue)(nil), // 2: google.protobuf.StringValue + (*wrapperspb.UInt64Value)(nil), // 3: google.protobuf.UInt64Value + (*wrapperspb.BoolValue)(nil), // 4: google.protobuf.BoolValue +} +var file_pkg_proto_eth_v1_validator_proto_depIdxs = []int32{ + 2, // 0: xatu.eth.v1.ValidatorData.pubkey:type_name -> google.protobuf.StringValue + 2, // 1: xatu.eth.v1.ValidatorData.withdrawal_credentials:type_name -> google.protobuf.StringValue + 3, // 2: xatu.eth.v1.ValidatorData.effective_balance:type_name -> google.protobuf.UInt64Value + 4, // 3: xatu.eth.v1.ValidatorData.slashed:type_name -> google.protobuf.BoolValue + 3, // 4: xatu.eth.v1.ValidatorData.activation_eligibility_epoch:type_name -> google.protobuf.UInt64Value + 3, // 5: xatu.eth.v1.ValidatorData.activation_epoch:type_name -> google.protobuf.UInt64Value + 3, // 6: xatu.eth.v1.ValidatorData.exit_epoch:type_name -> google.protobuf.UInt64Value + 3, // 7: xatu.eth.v1.ValidatorData.withdrawable_epoch:type_name -> google.protobuf.UInt64Value + 0, // 8: xatu.eth.v1.Validator.data:type_name -> xatu.eth.v1.ValidatorData + 3, // 9: xatu.eth.v1.Validator.index:type_name -> google.protobuf.UInt64Value + 3, // 10: xatu.eth.v1.Validator.balance:type_name -> google.protobuf.UInt64Value + 2, // 11: xatu.eth.v1.Validator.status:type_name -> google.protobuf.StringValue + 12, // [12:12] is the sub-list for method output_type + 12, // [12:12] is the sub-list for method input_type + 12, // [12:12] is the sub-list for extension type_name + 12, // [12:12] is the sub-list for extension extendee + 0, // [0:12] is the sub-list for field type_name +} + +func init() { file_pkg_proto_eth_v1_validator_proto_init() } +func file_pkg_proto_eth_v1_validator_proto_init() { + if File_pkg_proto_eth_v1_validator_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_pkg_proto_eth_v1_validator_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ValidatorData); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_proto_eth_v1_validator_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*Validator); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_pkg_proto_eth_v1_validator_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_pkg_proto_eth_v1_validator_proto_goTypes, + DependencyIndexes: file_pkg_proto_eth_v1_validator_proto_depIdxs, + MessageInfos: file_pkg_proto_eth_v1_validator_proto_msgTypes, + }.Build() + File_pkg_proto_eth_v1_validator_proto = out.File + file_pkg_proto_eth_v1_validator_proto_rawDesc = nil + file_pkg_proto_eth_v1_validator_proto_goTypes = nil + file_pkg_proto_eth_v1_validator_proto_depIdxs = nil +} diff --git a/pkg/proto/eth/v1/validator.proto b/pkg/proto/eth/v1/validator.proto new file mode 100644 index 00000000..adcbd473 --- /dev/null +++ b/pkg/proto/eth/v1/validator.proto @@ -0,0 +1,26 @@ +syntax = "proto3"; + +package xatu.eth.v1; + +option go_package = "github.com/ethpandaops/xatu/pkg/proto/eth/v1"; + +import "google/protobuf/wrappers.proto"; + +message ValidatorData { + google.protobuf.StringValue pubkey = 1; + google.protobuf.StringValue withdrawal_credentials = 2 [ json_name = "withdrawal_credentials" ]; + google.protobuf.UInt64Value effective_balance = 3 [ json_name = "effective_balance" ]; + google.protobuf.BoolValue slashed = 4; + google.protobuf.UInt64Value activation_eligibility_epoch = 5 [ json_name = "activation_eligibility_epoch" ]; + google.protobuf.UInt64Value activation_epoch = 6 [ json_name = "activation_epoch" ]; + google.protobuf.UInt64Value exit_epoch = 7 [ json_name = "exit_epoch" ]; + google.protobuf.UInt64Value withdrawable_epoch = 8 [ json_name = "withdrawable_epoch" ]; +} + +message Validator { + ValidatorData data = 1; + google.protobuf.UInt64Value index = 2; + google.protobuf.UInt64Value balance = 3; + google.protobuf.StringValue status = 4; +} + diff --git a/pkg/proto/eth/v2/beacon_block.pb.go b/pkg/proto/eth/v2/beacon_block.pb.go index 3b70d032..70987c38 100644 --- a/pkg/proto/eth/v2/beacon_block.pb.go +++ b/pkg/proto/eth/v2/beacon_block.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v2/beacon_block.proto // Note: largely inspired by diff --git a/pkg/proto/eth/v2/events.pb.go b/pkg/proto/eth/v2/events.pb.go index e89aa4f1..709a88d1 100644 --- a/pkg/proto/eth/v2/events.pb.go +++ b/pkg/proto/eth/v2/events.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v2/events.proto package v2 diff --git a/pkg/proto/eth/v2/withdrawals.pb.go b/pkg/proto/eth/v2/withdrawals.pb.go index 19ab86f8..c934b761 100644 --- a/pkg/proto/eth/v2/withdrawals.pb.go +++ b/pkg/proto/eth/v2/withdrawals.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/eth/v2/withdrawals.proto // Note: largely inspired by diff --git a/pkg/proto/libp2p/eth.pb.go b/pkg/proto/libp2p/eth.pb.go index 4c550365..80d817c0 100644 --- a/pkg/proto/libp2p/eth.pb.go +++ b/pkg/proto/libp2p/eth.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/libp2p/eth.proto package libp2p diff --git a/pkg/proto/libp2p/gossipsub/eth.pb.go b/pkg/proto/libp2p/gossipsub/eth.pb.go index 8e7da525..4ebdbede 100644 --- a/pkg/proto/libp2p/gossipsub/eth.pb.go +++ b/pkg/proto/libp2p/gossipsub/eth.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/libp2p/gossipsub/eth.proto package gossipsub diff --git a/pkg/proto/libp2p/peer.pb.go b/pkg/proto/libp2p/peer.pb.go index 6d6eec44..5770fb45 100644 --- a/pkg/proto/libp2p/peer.pb.go +++ b/pkg/proto/libp2p/peer.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/libp2p/peer.proto package libp2p diff --git a/pkg/proto/libp2p/trace.pb.go b/pkg/proto/libp2p/trace.pb.go index 59bb9d5d..0063ba32 100644 --- a/pkg/proto/libp2p/trace.pb.go +++ b/pkg/proto/libp2p/trace.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/libp2p/trace.proto package libp2p diff --git a/pkg/proto/xatu/coordinator.pb.go b/pkg/proto/xatu/coordinator.pb.go index 9b4dab36..fa55b731 100644 --- a/pkg/proto/xatu/coordinator.pb.go +++ b/pkg/proto/xatu/coordinator.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/xatu/coordinator.proto package xatu @@ -35,6 +35,7 @@ const ( CannonType_BEACON_API_ETH_V1_BEACON_BLOB_SIDECAR CannonType = 9 CannonType_BEACON_API_ETH_V1_PROPOSER_DUTY CannonType = 10 CannonType_BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION CannonType = 11 + CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS CannonType = 12 ) // Enum value maps for CannonType. @@ -52,6 +53,7 @@ var ( 9: "BEACON_API_ETH_V1_BEACON_BLOB_SIDECAR", 10: "BEACON_API_ETH_V1_PROPOSER_DUTY", 11: "BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION", + 12: "BEACON_API_ETH_V1_BEACON_VALIDATORS", } CannonType_value = map[string]int32{ "BEACON_API_ETH_V2_BEACON_BLOCK_VOLUNTARY_EXIT": 0, @@ -66,6 +68,7 @@ var ( "BEACON_API_ETH_V1_BEACON_BLOB_SIDECAR": 9, "BEACON_API_ETH_V1_PROPOSER_DUTY": 10, "BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION": 11, + "BEACON_API_ETH_V1_BEACON_VALIDATORS": 12, } ) @@ -1405,6 +1408,53 @@ func (x *CannonLocationEthV2BeaconBlockElaboratedAttestation) GetBackfillingChec return nil } +type CannonLocationEthV1BeaconValidators struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + BackfillingCheckpointMarker *BackfillingCheckpointMarker `protobuf:"bytes,1,opt,name=backfilling_checkpoint_marker,json=backfillingCheckpointMarker,proto3" json:"backfilling_checkpoint_marker,omitempty"` +} + +func (x *CannonLocationEthV1BeaconValidators) Reset() { + *x = CannonLocationEthV1BeaconValidators{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *CannonLocationEthV1BeaconValidators) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*CannonLocationEthV1BeaconValidators) ProtoMessage() {} + +func (x *CannonLocationEthV1BeaconValidators) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[25] + 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 CannonLocationEthV1BeaconValidators.ProtoReflect.Descriptor instead. +func (*CannonLocationEthV1BeaconValidators) Descriptor() ([]byte, []int) { + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{25} +} + +func (x *CannonLocationEthV1BeaconValidators) GetBackfillingCheckpointMarker() *BackfillingCheckpointMarker { + if x != nil { + return x.BackfillingCheckpointMarker + } + return nil +} + type CannonLocation struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1426,13 +1476,14 @@ type CannonLocation struct { // *CannonLocation_EthV1BeaconBlobSidecar // *CannonLocation_EthV1BeaconProposerDuty // *CannonLocation_EthV2BeaconBlockElaboratedAttestation + // *CannonLocation_EthV1BeaconValidators Data isCannonLocation_Data `protobuf_oneof:"Data"` } func (x *CannonLocation) Reset() { *x = CannonLocation{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[25] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1445,7 +1496,7 @@ func (x *CannonLocation) String() string { func (*CannonLocation) ProtoMessage() {} func (x *CannonLocation) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[25] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1458,7 +1509,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{25} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{26} } func (x *CannonLocation) GetNetworkId() string { @@ -1566,6 +1617,13 @@ func (x *CannonLocation) GetEthV2BeaconBlockElaboratedAttestation() *CannonLocat return nil } +func (x *CannonLocation) GetEthV1BeaconValidators() *CannonLocationEthV1BeaconValidators { + if x, ok := x.GetData().(*CannonLocation_EthV1BeaconValidators); ok { + return x.EthV1BeaconValidators + } + return nil +} + type isCannonLocation_Data interface { isCannonLocation_Data() } @@ -1618,6 +1676,10 @@ type CannonLocation_EthV2BeaconBlockElaboratedAttestation struct { EthV2BeaconBlockElaboratedAttestation *CannonLocationEthV2BeaconBlockElaboratedAttestation `protobuf:"bytes,14,opt,name=eth_v2_beacon_block_elaborated_attestation,json=BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION,proto3,oneof"` } +type CannonLocation_EthV1BeaconValidators struct { + EthV1BeaconValidators *CannonLocationEthV1BeaconValidators `protobuf:"bytes,15,opt,name=eth_v1_beacon_validators,json=BEACON_API_ETH_V1_BEACON_VALIDATORS,proto3,oneof"` +} + func (*CannonLocation_EthV2BeaconBlockVoluntaryExit) isCannonLocation_Data() {} func (*CannonLocation_EthV2BeaconBlockProposerSlashing) isCannonLocation_Data() {} @@ -1642,6 +1704,8 @@ func (*CannonLocation_EthV1BeaconProposerDuty) isCannonLocation_Data() {} func (*CannonLocation_EthV2BeaconBlockElaboratedAttestation) isCannonLocation_Data() {} +func (*CannonLocation_EthV1BeaconValidators) isCannonLocation_Data() {} + type GetCannonLocationRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1654,7 +1718,7 @@ type GetCannonLocationRequest struct { func (x *GetCannonLocationRequest) Reset() { *x = GetCannonLocationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[26] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1667,7 +1731,7 @@ func (x *GetCannonLocationRequest) String() string { func (*GetCannonLocationRequest) ProtoMessage() {} func (x *GetCannonLocationRequest) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[26] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1680,7 +1744,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{26} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{27} } func (x *GetCannonLocationRequest) GetNetworkId() string { @@ -1708,7 +1772,7 @@ type GetCannonLocationResponse struct { func (x *GetCannonLocationResponse) Reset() { *x = GetCannonLocationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[27] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1721,7 +1785,7 @@ func (x *GetCannonLocationResponse) String() string { func (*GetCannonLocationResponse) ProtoMessage() {} func (x *GetCannonLocationResponse) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[27] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1734,7 +1798,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{27} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{28} } func (x *GetCannonLocationResponse) GetLocation() *CannonLocation { @@ -1755,7 +1819,7 @@ type UpsertCannonLocationRequest struct { func (x *UpsertCannonLocationRequest) Reset() { *x = UpsertCannonLocationRequest{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[28] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1768,7 +1832,7 @@ func (x *UpsertCannonLocationRequest) String() string { func (*UpsertCannonLocationRequest) ProtoMessage() {} func (x *UpsertCannonLocationRequest) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[28] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1781,7 +1845,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{28} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{29} } func (x *UpsertCannonLocationRequest) GetLocation() *CannonLocation { @@ -1800,7 +1864,7 @@ type UpsertCannonLocationResponse struct { func (x *UpsertCannonLocationResponse) Reset() { *x = UpsertCannonLocationResponse{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[29] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1813,7 +1877,7 @@ func (x *UpsertCannonLocationResponse) String() string { func (*UpsertCannonLocationResponse) ProtoMessage() {} func (x *UpsertCannonLocationResponse) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[29] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1826,7 +1890,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{29} + return file_pkg_proto_xatu_coordinator_proto_rawDescGZIP(), []int{30} } type ExecutionNodeStatus_Capability struct { @@ -1841,7 +1905,7 @@ type ExecutionNodeStatus_Capability struct { func (x *ExecutionNodeStatus_Capability) Reset() { *x = ExecutionNodeStatus_Capability{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[30] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1854,7 +1918,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[30] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1896,7 +1960,7 @@ type ExecutionNodeStatus_ForkID struct { func (x *ExecutionNodeStatus_ForkID) Reset() { *x = ExecutionNodeStatus_ForkID{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[31] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1909,7 +1973,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[31] + mi := &file_pkg_proto_xatu_coordinator_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2111,226 +2175,245 @@ var file_pkg_proto_xatu_coordinator_proto_rawDesc = []byte{ 0x78, 0x61, 0x74, 0x75, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x1b, 0x62, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x22, 0xef, 0x0d, - 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, + 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x22, 0x8c, 0x01, + 0x0a, 0x23, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x56, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x65, 0x0a, 0x1d, 0x62, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, + 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, + 0x6d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, + 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x52, + 0x1b, 0x62, 0x61, 0x63, 0x6b, 0x66, 0x69, 0x6c, 0x6c, 0x69, 0x6e, 0x67, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x4d, 0x61, 0x72, 0x6b, 0x65, 0x72, 0x22, 0xe3, 0x0e, 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, 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, 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, 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, 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, + 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, 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, 0x12, 0x83, 0x01, 0x0a, 0x1e, 0x65, 0x74, 0x68, 0x5f, 0x76, - 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x77, - 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2e, 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, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x48, - 0x00, 0x52, 0x29, 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, 0x57, 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, 0x4c, 0x12, 0x63, 0x0a, 0x13, + 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, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 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, 0x7d, 0x0a, 0x1f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x5f, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0b, 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, - 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, - 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, - 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, 0x4c, 0x4f, 0x43, - 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, - 0x12, 0x77, 0x0a, 0x1a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, - 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x0c, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, - 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, - 0x48, 0x00, 0x52, 0x25, 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, 0x42, 0x4c, 0x4f, - 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x73, 0x0a, 0x1b, 0x65, 0x74, 0x68, - 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, - 0x73, 0x65, 0x72, 0x5f, 0x64, 0x75, 0x74, 0x79, 0x18, 0x0d, 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, 0x31, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x50, - 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, 0x75, 0x74, 0x79, 0x48, 0x00, 0x52, 0x1f, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, - 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, 0x12, 0xa6, - 0x01, 0x0a, 0x2a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, - 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, - 0x64, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0e, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, + 0x6f, 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, 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, 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, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, - 0x74, 0x65, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, - 0x52, 0x35, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, + 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, 0x12, 0x83, 0x01, 0x0a, 0x1e, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, + 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x77, 0x69, + 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, + 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, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x48, 0x00, + 0x52, 0x29, 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, 0x57, 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, 0x4c, 0x12, 0x63, 0x0a, 0x13, 0x65, + 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, + 0x63, 0x6b, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 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, - 0x5f, 0x45, 0x4c, 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x54, 0x54, 0x45, - 0x53, 0x54, 0x41, 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, + 0x12, 0x7d, 0x0a, 0x1f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x5f, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x0b, 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, 0x42, + 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6c, + 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x1f, + 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, + 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, + 0x77, 0x0a, 0x1a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, + 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x0c, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x61, 0x6e, 0x6e, 0x6f, + 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, 0x65, + 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x48, + 0x00, 0x52, 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, + 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x73, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, + 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, + 0x65, 0x72, 0x5f, 0x64, 0x75, 0x74, 0x79, 0x18, 0x0d, 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, 0x31, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x50, 0x72, + 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, 0x75, 0x74, 0x79, 0x48, 0x00, 0x52, 0x1f, 0x42, 0x45, + 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, + 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, 0x12, 0xa6, 0x01, + 0x0a, 0x2a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, + 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x39, 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, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, + 0x65, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, + 0x35, 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, 0x4c, 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, + 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x72, 0x0a, 0x18, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, + 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, + 0x72, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x74, + 0x68, 0x56, 0x31, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x6f, 0x72, 0x73, 0x48, 0x00, 0x52, 0x23, 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, + 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, 0x52, 0x53, 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, - 0x2a, 0xd0, 0x04, 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, + 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, 0xf9, 0x04, 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, 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, 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, - 0x12, 0x2d, 0x0a, 0x29, 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, 0x57, 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, 0x4c, 0x10, 0x06, 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, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, 0x52, 0x49, 0x4e, - 0x54, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, 0x49, 0x46, 0x49, - 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x29, 0x0a, 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, - 0x52, 0x10, 0x09, 0x12, 0x23, 0x0a, 0x1f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, - 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, - 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, 0x10, 0x0a, 0x12, 0x39, 0x0a, 0x35, 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, 0x4c, 0x41, 0x42, 0x4f, - 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, - 0x4e, 0x10, 0x0b, 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, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x1f, - 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, + 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, 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, 0x12, 0x2d, 0x0a, 0x29, 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, 0x57, 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, 0x4c, + 0x10, 0x06, 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, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, + 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, + 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x08, 0x12, 0x29, 0x0a, 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, + 0x45, 0x43, 0x41, 0x52, 0x10, 0x09, 0x12, 0x23, 0x0a, 0x1f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, + 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x50, 0x52, 0x4f, 0x50, + 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, 0x10, 0x0a, 0x12, 0x39, 0x0a, 0x35, 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, 0x4c, + 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x0b, 0x12, 0x27, 0x0a, 0x23, 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, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, 0x52, 0x53, 0x10, 0x0c, 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, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x1f, 0x4c, 0x69, 0x73, 0x74, + 0x53, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x12, 0x2c, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, 0x45, 0x78, + 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, + 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x1f, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, + 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2c, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7d, 0x0a, + 0x1e, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x12, - 0x2c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x6c, 0x6c, - 0x65, 0x64, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x6c, 0x6c, 0x65, 0x64, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, - 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, - 0x01, 0x0a, 0x1f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x2c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, - 0x6f, 0x72, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2d, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, + 0x2b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, + 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, + 0x63, 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, - 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x7d, 0x0a, 0x1e, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x45, - 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, - 0x72, 0x64, 0x73, 0x12, 0x2b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6f, 0x6f, 0x72, 0x64, - 0x69, 0x6e, 0x61, 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, - 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, - 0x74, 0x65, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x6f, 0x64, 0x65, 0x52, - 0x65, 0x63, 0x6f, 0x72, 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x65, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, - 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x23, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x4e, 0x6f, - 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x24, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, - 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x56, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x43, 0x61, - 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 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, 0x22, 0x00, 0x12, - 0x5f, 0x0a, 0x14, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, - 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x55, + 0x64, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x65, 0x0a, 0x16, + 0x47, 0x65, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, + 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x12, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x47, 0x65, + 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, + 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x4e, + 0x6f, 0x64, 0x65, 0x52, 0x65, 0x63, 0x6f, 0x72, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x56, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, + 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x47, 0x65, 0x74, 0x43, 0x61, 0x6e, 0x6e, 0x6f, 0x6e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 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, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x14, 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, 0x1a, 0x22, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 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, 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, + 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 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, 0x1a, 0x22, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 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, 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 ( @@ -2346,7 +2429,7 @@ 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, 32) +var file_pkg_proto_xatu_coordinator_proto_msgTypes = make([]protoimpl.MessageInfo, 33) var file_pkg_proto_xatu_coordinator_proto_goTypes = []interface{}{ (CannonType)(0), // 0: xatu.CannonType (*CreateNodeRecordsRequest)(nil), // 1: xatu.CreateNodeRecordsRequest @@ -2374,56 +2457,59 @@ var file_pkg_proto_xatu_coordinator_proto_goTypes = []interface{}{ (*CannonLocationEthV1BeaconBlobSidecar)(nil), // 23: xatu.CannonLocationEthV1BeaconBlobSidecar (*CannonLocationEthV1BeaconProposerDuty)(nil), // 24: xatu.CannonLocationEthV1BeaconProposerDuty (*CannonLocationEthV2BeaconBlockElaboratedAttestation)(nil), // 25: xatu.CannonLocationEthV2BeaconBlockElaboratedAttestation - (*CannonLocation)(nil), // 26: xatu.CannonLocation - (*GetCannonLocationRequest)(nil), // 27: xatu.GetCannonLocationRequest - (*GetCannonLocationResponse)(nil), // 28: xatu.GetCannonLocationResponse - (*UpsertCannonLocationRequest)(nil), // 29: xatu.UpsertCannonLocationRequest - (*UpsertCannonLocationResponse)(nil), // 30: xatu.UpsertCannonLocationResponse - (*ExecutionNodeStatus_Capability)(nil), // 31: xatu.ExecutionNodeStatus.Capability - (*ExecutionNodeStatus_ForkID)(nil), // 32: xatu.ExecutionNodeStatus.ForkID + (*CannonLocationEthV1BeaconValidators)(nil), // 26: xatu.CannonLocationEthV1BeaconValidators + (*CannonLocation)(nil), // 27: xatu.CannonLocation + (*GetCannonLocationRequest)(nil), // 28: xatu.GetCannonLocationRequest + (*GetCannonLocationResponse)(nil), // 29: xatu.GetCannonLocationResponse + (*UpsertCannonLocationRequest)(nil), // 30: xatu.UpsertCannonLocationRequest + (*UpsertCannonLocationResponse)(nil), // 31: xatu.UpsertCannonLocationResponse + (*ExecutionNodeStatus_Capability)(nil), // 32: xatu.ExecutionNodeStatus.Capability + (*ExecutionNodeStatus_ForkID)(nil), // 33: xatu.ExecutionNodeStatus.ForkID } var file_pkg_proto_xatu_coordinator_proto_depIdxs = []int32{ - 31, // 0: xatu.ExecutionNodeStatus.capabilities:type_name -> xatu.ExecutionNodeStatus.Capability - 32, // 1: xatu.ExecutionNodeStatus.fork_id:type_name -> xatu.ExecutionNodeStatus.ForkID + 32, // 0: xatu.ExecutionNodeStatus.capabilities:type_name -> xatu.ExecutionNodeStatus.Capability + 33, // 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 13, // 4: xatu.CannonLocationEthV1BeaconProposerDuty.backfilling_checkpoint_marker:type_name -> xatu.BackfillingCheckpointMarker 13, // 5: xatu.CannonLocationEthV2BeaconBlockElaboratedAttestation.backfilling_checkpoint_marker:type_name -> xatu.BackfillingCheckpointMarker - 0, // 6: xatu.CannonLocation.type:type_name -> xatu.CannonType - 14, // 7: xatu.CannonLocation.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.CannonLocationEthV2BeaconBlockVoluntaryExit - 15, // 8: xatu.CannonLocation.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.CannonLocationEthV2BeaconBlockProposerSlashing - 16, // 9: xatu.CannonLocation.eth_v2_beacon_block_deposit:type_name -> xatu.CannonLocationEthV2BeaconBlockDeposit - 17, // 10: xatu.CannonLocation.eth_v2_beacon_block_attester_slashing:type_name -> xatu.CannonLocationEthV2BeaconBlockAttesterSlashing - 18, // 11: xatu.CannonLocation.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.CannonLocationEthV2BeaconBlockBlsToExecutionChange - 19, // 12: xatu.CannonLocation.eth_v2_beacon_block_execution_transaction:type_name -> xatu.CannonLocationEthV2BeaconBlockExecutionTransaction - 20, // 13: xatu.CannonLocation.eth_v2_beacon_block_withdrawal:type_name -> xatu.CannonLocationEthV2BeaconBlockWithdrawal - 21, // 14: xatu.CannonLocation.eth_v2_beacon_block:type_name -> xatu.CannonLocationEthV2BeaconBlock - 22, // 15: xatu.CannonLocation.blockprint_block_classification:type_name -> xatu.CannonLocationBlockprintBlockClassification - 23, // 16: xatu.CannonLocation.eth_v1_beacon_blob_sidecar:type_name -> xatu.CannonLocationEthV1BeaconBlobSidecar - 24, // 17: xatu.CannonLocation.eth_v1_beacon_proposer_duty:type_name -> xatu.CannonLocationEthV1BeaconProposerDuty - 25, // 18: xatu.CannonLocation.eth_v2_beacon_block_elaborated_attestation:type_name -> xatu.CannonLocationEthV2BeaconBlockElaboratedAttestation - 0, // 19: xatu.GetCannonLocationRequest.type:type_name -> xatu.CannonType - 26, // 20: xatu.GetCannonLocationResponse.location:type_name -> xatu.CannonLocation - 26, // 21: xatu.UpsertCannonLocationRequest.location:type_name -> xatu.CannonLocation - 1, // 22: xatu.Coordinator.CreateNodeRecords:input_type -> xatu.CreateNodeRecordsRequest - 3, // 23: xatu.Coordinator.ListStalledExecutionNodeRecords:input_type -> xatu.ListStalledExecutionNodeRecordsRequest - 6, // 24: xatu.Coordinator.CreateExecutionNodeRecordStatus:input_type -> xatu.CreateExecutionNodeRecordStatusRequest - 9, // 25: xatu.Coordinator.CoordinateExecutionNodeRecords:input_type -> xatu.CoordinateExecutionNodeRecordsRequest - 11, // 26: xatu.Coordinator.GetDiscoveryNodeRecord:input_type -> xatu.GetDiscoveryNodeRecordRequest - 27, // 27: xatu.Coordinator.GetCannonLocation:input_type -> xatu.GetCannonLocationRequest - 29, // 28: xatu.Coordinator.UpsertCannonLocation:input_type -> xatu.UpsertCannonLocationRequest - 2, // 29: xatu.Coordinator.CreateNodeRecords:output_type -> xatu.CreateNodeRecordsResponse - 4, // 30: xatu.Coordinator.ListStalledExecutionNodeRecords:output_type -> xatu.ListStalledExecutionNodeRecordsResponse - 7, // 31: xatu.Coordinator.CreateExecutionNodeRecordStatus:output_type -> xatu.CreateExecutionNodeRecordStatusResponse - 10, // 32: xatu.Coordinator.CoordinateExecutionNodeRecords:output_type -> xatu.CoordinateExecutionNodeRecordsResponse - 12, // 33: xatu.Coordinator.GetDiscoveryNodeRecord:output_type -> xatu.GetDiscoveryNodeRecordResponse - 28, // 34: xatu.Coordinator.GetCannonLocation:output_type -> xatu.GetCannonLocationResponse - 30, // 35: xatu.Coordinator.UpsertCannonLocation:output_type -> xatu.UpsertCannonLocationResponse - 29, // [29:36] is the sub-list for method output_type - 22, // [22:29] is the sub-list for method input_type - 22, // [22:22] is the sub-list for extension type_name - 22, // [22:22] is the sub-list for extension extendee - 0, // [0:22] is the sub-list for field type_name + 13, // 6: xatu.CannonLocationEthV1BeaconValidators.backfilling_checkpoint_marker:type_name -> xatu.BackfillingCheckpointMarker + 0, // 7: xatu.CannonLocation.type:type_name -> xatu.CannonType + 14, // 8: xatu.CannonLocation.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.CannonLocationEthV2BeaconBlockVoluntaryExit + 15, // 9: xatu.CannonLocation.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.CannonLocationEthV2BeaconBlockProposerSlashing + 16, // 10: xatu.CannonLocation.eth_v2_beacon_block_deposit:type_name -> xatu.CannonLocationEthV2BeaconBlockDeposit + 17, // 11: xatu.CannonLocation.eth_v2_beacon_block_attester_slashing:type_name -> xatu.CannonLocationEthV2BeaconBlockAttesterSlashing + 18, // 12: xatu.CannonLocation.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.CannonLocationEthV2BeaconBlockBlsToExecutionChange + 19, // 13: xatu.CannonLocation.eth_v2_beacon_block_execution_transaction:type_name -> xatu.CannonLocationEthV2BeaconBlockExecutionTransaction + 20, // 14: xatu.CannonLocation.eth_v2_beacon_block_withdrawal:type_name -> xatu.CannonLocationEthV2BeaconBlockWithdrawal + 21, // 15: xatu.CannonLocation.eth_v2_beacon_block:type_name -> xatu.CannonLocationEthV2BeaconBlock + 22, // 16: xatu.CannonLocation.blockprint_block_classification:type_name -> xatu.CannonLocationBlockprintBlockClassification + 23, // 17: xatu.CannonLocation.eth_v1_beacon_blob_sidecar:type_name -> xatu.CannonLocationEthV1BeaconBlobSidecar + 24, // 18: xatu.CannonLocation.eth_v1_beacon_proposer_duty:type_name -> xatu.CannonLocationEthV1BeaconProposerDuty + 25, // 19: xatu.CannonLocation.eth_v2_beacon_block_elaborated_attestation:type_name -> xatu.CannonLocationEthV2BeaconBlockElaboratedAttestation + 26, // 20: xatu.CannonLocation.eth_v1_beacon_validators:type_name -> xatu.CannonLocationEthV1BeaconValidators + 0, // 21: xatu.GetCannonLocationRequest.type:type_name -> xatu.CannonType + 27, // 22: xatu.GetCannonLocationResponse.location:type_name -> xatu.CannonLocation + 27, // 23: xatu.UpsertCannonLocationRequest.location:type_name -> xatu.CannonLocation + 1, // 24: xatu.Coordinator.CreateNodeRecords:input_type -> xatu.CreateNodeRecordsRequest + 3, // 25: xatu.Coordinator.ListStalledExecutionNodeRecords:input_type -> xatu.ListStalledExecutionNodeRecordsRequest + 6, // 26: xatu.Coordinator.CreateExecutionNodeRecordStatus:input_type -> xatu.CreateExecutionNodeRecordStatusRequest + 9, // 27: xatu.Coordinator.CoordinateExecutionNodeRecords:input_type -> xatu.CoordinateExecutionNodeRecordsRequest + 11, // 28: xatu.Coordinator.GetDiscoveryNodeRecord:input_type -> xatu.GetDiscoveryNodeRecordRequest + 28, // 29: xatu.Coordinator.GetCannonLocation:input_type -> xatu.GetCannonLocationRequest + 30, // 30: xatu.Coordinator.UpsertCannonLocation:input_type -> xatu.UpsertCannonLocationRequest + 2, // 31: xatu.Coordinator.CreateNodeRecords:output_type -> xatu.CreateNodeRecordsResponse + 4, // 32: xatu.Coordinator.ListStalledExecutionNodeRecords:output_type -> xatu.ListStalledExecutionNodeRecordsResponse + 7, // 33: xatu.Coordinator.CreateExecutionNodeRecordStatus:output_type -> xatu.CreateExecutionNodeRecordStatusResponse + 10, // 34: xatu.Coordinator.CoordinateExecutionNodeRecords:output_type -> xatu.CoordinateExecutionNodeRecordsResponse + 12, // 35: xatu.Coordinator.GetDiscoveryNodeRecord:output_type -> xatu.GetDiscoveryNodeRecordResponse + 29, // 36: xatu.Coordinator.GetCannonLocation:output_type -> xatu.GetCannonLocationResponse + 31, // 37: xatu.Coordinator.UpsertCannonLocation:output_type -> xatu.UpsertCannonLocationResponse + 31, // [31:38] is the sub-list for method output_type + 24, // [24:31] is the sub-list for method input_type + 24, // [24:24] is the sub-list for extension type_name + 24, // [24:24] is the sub-list for extension extendee + 0, // [0:24] is the sub-list for field type_name } func init() { file_pkg_proto_xatu_coordinator_proto_init() } @@ -2733,7 +2819,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CannonLocation); i { + switch v := v.(*CannonLocationEthV1BeaconValidators); i { case 0: return &v.state case 1: @@ -2745,7 +2831,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetCannonLocationRequest); i { + switch v := v.(*CannonLocation); i { case 0: return &v.state case 1: @@ -2757,7 +2843,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetCannonLocationResponse); i { + switch v := v.(*GetCannonLocationRequest); i { case 0: return &v.state case 1: @@ -2769,7 +2855,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpsertCannonLocationRequest); i { + switch v := v.(*GetCannonLocationResponse); i { case 0: return &v.state case 1: @@ -2781,7 +2867,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpsertCannonLocationResponse); i { + switch v := v.(*UpsertCannonLocationRequest); i { case 0: return &v.state case 1: @@ -2793,7 +2879,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ExecutionNodeStatus_Capability); i { + switch v := v.(*UpsertCannonLocationResponse); i { case 0: return &v.state case 1: @@ -2805,6 +2891,18 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } file_pkg_proto_xatu_coordinator_proto_msgTypes[31].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[32].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ExecutionNodeStatus_ForkID); i { case 0: return &v.state @@ -2817,7 +2915,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { } } } - file_pkg_proto_xatu_coordinator_proto_msgTypes[25].OneofWrappers = []interface{}{ + file_pkg_proto_xatu_coordinator_proto_msgTypes[26].OneofWrappers = []interface{}{ (*CannonLocation_EthV2BeaconBlockVoluntaryExit)(nil), (*CannonLocation_EthV2BeaconBlockProposerSlashing)(nil), (*CannonLocation_EthV2BeaconBlockDeposit)(nil), @@ -2830,6 +2928,7 @@ func file_pkg_proto_xatu_coordinator_proto_init() { (*CannonLocation_EthV1BeaconBlobSidecar)(nil), (*CannonLocation_EthV1BeaconProposerDuty)(nil), (*CannonLocation_EthV2BeaconBlockElaboratedAttestation)(nil), + (*CannonLocation_EthV1BeaconValidators)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -2837,7 +2936,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: 32, + NumMessages: 33, NumExtensions: 0, NumServices: 1, }, diff --git a/pkg/proto/xatu/coordinator.proto b/pkg/proto/xatu/coordinator.proto index 196cfdd1..383c82a5 100644 --- a/pkg/proto/xatu/coordinator.proto +++ b/pkg/proto/xatu/coordinator.proto @@ -100,6 +100,7 @@ enum CannonType { BEACON_API_ETH_V1_BEACON_BLOB_SIDECAR = 9; BEACON_API_ETH_V1_PROPOSER_DUTY = 10; BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION = 11; + BEACON_API_ETH_V1_BEACON_VALIDATORS = 12; } message BackfillingCheckpointMarker { @@ -138,6 +139,10 @@ message CannonLocationEthV2BeaconBlockElaboratedAttestation { BackfillingCheckpointMarker backfilling_checkpoint_marker = 1; } +message CannonLocationEthV1BeaconValidators { + BackfillingCheckpointMarker backfilling_checkpoint_marker = 1; +} + message CannonLocation { string network_id = 1; CannonType type = 2; @@ -174,6 +179,8 @@ message CannonLocation { CannonLocationEthV2BeaconBlockElaboratedAttestation eth_v2_beacon_block_elaborated_attestation = 14 [ json_name = "BEACON_API_ETH_V2_BEACON_BLOCK_ELABORATED_ATTESTATION" ]; + CannonLocationEthV1BeaconValidators eth_v1_beacon_validators = 15 + [ json_name = "BEACON_API_ETH_V1_BEACON_VALIDATORS" ]; } } diff --git a/pkg/proto/xatu/coordinator_grpc.pb.go b/pkg/proto/xatu/coordinator_grpc.pb.go index 62dd61d1..2bfbdc1d 100644 --- a/pkg/proto/xatu/coordinator_grpc.pb.go +++ b/pkg/proto/xatu/coordinator_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.3.0 -// - protoc v4.25.3 +// - protoc v4.24.2 // source: pkg/proto/xatu/coordinator.proto package xatu diff --git a/pkg/proto/xatu/event_ingester.pb.go b/pkg/proto/xatu/event_ingester.pb.go index 7562b9ce..bdfa78d4 100644 --- a/pkg/proto/xatu/event_ingester.pb.go +++ b/pkg/proto/xatu/event_ingester.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.32.0 -// protoc v4.25.3 +// protoc-gen-go v1.31.0 +// protoc v4.24.2 // source: pkg/proto/xatu/event_ingester.proto package xatu @@ -82,6 +82,7 @@ const ( Event_LIBP2P_TRACE_GOSSIPSUB_BEACON_BLOCK Event_Name = 49 Event_LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION Event_Name = 50 Event_LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR Event_Name = 51 + Event_BEACON_API_ETH_V1_BEACON_VALIDATORS Event_Name = 52 ) // Enum value maps for Event_Name. @@ -139,6 +140,7 @@ var ( 49: "LIBP2P_TRACE_GOSSIPSUB_BEACON_BLOCK", 50: "LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION", 51: "LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR", + 52: "BEACON_API_ETH_V1_BEACON_VALIDATORS", } Event_Name_value = map[string]int32{ "BEACON_API_ETH_V1_EVENTS_UNKNOWN": 0, @@ -193,6 +195,7 @@ var ( "LIBP2P_TRACE_GOSSIPSUB_BEACON_BLOCK": 49, "LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION": 50, "LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR": 51, + "BEACON_API_ETH_V1_BEACON_VALIDATORS": 52, } ) @@ -220,7 +223,7 @@ func (x Event_Name) Number() protoreflect.EnumNumber { // Deprecated: Use Event_Name.Descriptor instead. func (Event_Name) EnumDescriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{17, 0} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{18, 0} } type CreateEventsRequest struct { @@ -929,6 +932,53 @@ func (x *DebugForkChoiceReorgV2) GetEvent() *v1.EventChainReorgV2 { return nil } +type Validators struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Validators []*v1.Validator `protobuf:"bytes,1,rep,name=validators,proto3" json:"validators,omitempty"` +} + +func (x *Validators) Reset() { + *x = Validators{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Validators) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Validators) ProtoMessage() {} + +func (x *Validators) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[13] + 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 Validators.ProtoReflect.Descriptor instead. +func (*Validators) Descriptor() ([]byte, []int) { + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{13} +} + +func (x *Validators) GetValidators() []*v1.Validator { + if x != nil { + return x.Validators + } + return nil +} + type BlockIdentifier struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -947,7 +997,7 @@ type BlockIdentifier struct { func (x *BlockIdentifier) Reset() { *x = BlockIdentifier{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[13] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -960,7 +1010,7 @@ func (x *BlockIdentifier) String() string { func (*BlockIdentifier) ProtoMessage() {} func (x *BlockIdentifier) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[13] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -973,7 +1023,7 @@ func (x *BlockIdentifier) ProtoReflect() protoreflect.Message { // Deprecated: Use BlockIdentifier.ProtoReflect.Descriptor instead. func (*BlockIdentifier) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{13} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14} } func (x *BlockIdentifier) GetEpoch() *EpochV2 { @@ -1079,13 +1129,14 @@ type ClientMeta struct { // *ClientMeta_Libp2PTraceGossipsubBeaconBlock // *ClientMeta_Libp2PTraceGossipsubBeaconAttestation // *ClientMeta_Libp2PTraceGossipsubBlobSidecar + // *ClientMeta_EthV1Validators AdditionalData isClientMeta_AdditionalData `protobuf_oneof:"AdditionalData"` } func (x *ClientMeta) Reset() { *x = ClientMeta{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[14] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1098,7 +1149,7 @@ func (x *ClientMeta) String() string { func (*ClientMeta) ProtoMessage() {} func (x *ClientMeta) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[14] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1111,7 +1162,7 @@ func (x *ClientMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta.ProtoReflect.Descriptor instead. func (*ClientMeta) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15} } func (x *ClientMeta) GetName() string { @@ -1520,6 +1571,13 @@ func (x *ClientMeta) GetLibp2PTraceGossipsubBlobSidecar() *ClientMeta_Additional return nil } +func (x *ClientMeta) GetEthV1Validators() *ClientMeta_AdditionalEthV1ValidatorsData { + if x, ok := x.GetAdditionalData().(*ClientMeta_EthV1Validators); ok { + return x.EthV1Validators + } + return nil +} + type isClientMeta_AdditionalData interface { isClientMeta_AdditionalData() } @@ -1798,6 +1856,11 @@ type ClientMeta_Libp2PTraceGossipsubBlobSidecar struct { Libp2PTraceGossipsubBlobSidecar *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData `protobuf:"bytes,59,opt,name=libp2p_trace_gossipsub_blob_sidecar,json=LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR,proto3,oneof"` } +type ClientMeta_EthV1Validators struct { + // AdditionalEthV1ValidatorsData contains additional data about the eth v1 validators. + EthV1Validators *ClientMeta_AdditionalEthV1ValidatorsData `protobuf:"bytes,60,opt,name=eth_v1_validators,json=BEACON_API_ETH_V1_BEACON_VALIDATORS,proto3,oneof"` +} + func (*ClientMeta_EthV1EventsAttestation) isClientMeta_AdditionalData() {} func (*ClientMeta_EthV1EventsHead) isClientMeta_AdditionalData() {} @@ -1896,6 +1959,8 @@ func (*ClientMeta_Libp2PTraceGossipsubBeaconAttestation) isClientMeta_Additional func (*ClientMeta_Libp2PTraceGossipsubBlobSidecar) isClientMeta_AdditionalData() {} +func (*ClientMeta_EthV1Validators) isClientMeta_AdditionalData() {} + type ServerMeta struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1919,7 +1984,7 @@ type ServerMeta struct { func (x *ServerMeta) Reset() { *x = ServerMeta{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[15] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1932,7 +1997,7 @@ func (x *ServerMeta) String() string { func (*ServerMeta) ProtoMessage() {} func (x *ServerMeta) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[15] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1945,7 +2010,7 @@ func (x *ServerMeta) ProtoReflect() protoreflect.Message { // Deprecated: Use ServerMeta.ProtoReflect.Descriptor instead. func (*ServerMeta) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16} } func (x *ServerMeta) GetEvent() *ServerMeta_Event { @@ -2032,7 +2097,7 @@ type Meta struct { func (x *Meta) Reset() { *x = Meta{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[16] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2045,7 +2110,7 @@ func (x *Meta) String() string { func (*Meta) ProtoMessage() {} func (x *Meta) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[16] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2058,7 +2123,7 @@ func (x *Meta) ProtoReflect() protoreflect.Message { // Deprecated: Use Meta.ProtoReflect.Descriptor instead. func (*Meta) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{17} } func (x *Meta) GetClient() *ClientMeta { @@ -2091,7 +2156,7 @@ type Event struct { func (x *Event) Reset() { *x = Event{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[17] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2104,7 +2169,7 @@ func (x *Event) String() string { func (*Event) ProtoMessage() {} func (x *Event) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[17] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2117,7 +2182,7 @@ func (x *Event) ProtoReflect() protoreflect.Message { // Deprecated: Use Event.ProtoReflect.Descriptor instead. func (*Event) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{17} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{18} } func (x *Event) GetName() Event_Name { @@ -2201,13 +2266,14 @@ type DecoratedEvent struct { // *DecoratedEvent_Libp2PTraceGossipsubBeaconBlock // *DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation // *DecoratedEvent_Libp2PTraceGossipsubBlobSidecar + // *DecoratedEvent_EthV1Validators Data isDecoratedEvent_Data `protobuf_oneof:"data"` } func (x *DecoratedEvent) Reset() { *x = DecoratedEvent{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[18] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2220,7 +2286,7 @@ func (x *DecoratedEvent) String() string { func (*DecoratedEvent) ProtoMessage() {} func (x *DecoratedEvent) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[18] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2233,7 +2299,7 @@ func (x *DecoratedEvent) ProtoReflect() protoreflect.Message { // Deprecated: Use DecoratedEvent.ProtoReflect.Descriptor instead. func (*DecoratedEvent) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{18} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{19} } func (x *DecoratedEvent) GetEvent() *Event { @@ -2611,6 +2677,13 @@ func (x *DecoratedEvent) GetLibp2PTraceGossipsubBlobSidecar() *gossipsub.BlobSid return nil } +func (x *DecoratedEvent) GetEthV1Validators() *Validators { + if x, ok := x.GetData().(*DecoratedEvent_EthV1Validators); ok { + return x.EthV1Validators + } + return nil +} + type isDecoratedEvent_Data interface { isDecoratedEvent_Data() } @@ -2822,6 +2895,10 @@ type DecoratedEvent_Libp2PTraceGossipsubBlobSidecar struct { Libp2PTraceGossipsubBlobSidecar *gossipsub.BlobSidecar `protobuf:"bytes,51,opt,name=libp2p_trace_gossipsub_blob_sidecar,json=LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR,proto3,oneof"` } +type DecoratedEvent_EthV1Validators struct { + EthV1Validators *Validators `protobuf:"bytes,52,opt,name=eth_v1_validators,json=BEACON_API_ETH_V1_BEACON_VALIDATORS,proto3,oneof"` +} + func (*DecoratedEvent_EthV1EventsAttestation) isDecoratedEvent_Data() {} func (*DecoratedEvent_EthV1EventsBlock) isDecoratedEvent_Data() {} @@ -2920,6 +2997,8 @@ func (*DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation) isDecoratedEvent_Da func (*DecoratedEvent_Libp2PTraceGossipsubBlobSidecar) isDecoratedEvent_Data() {} +func (*DecoratedEvent_EthV1Validators) isDecoratedEvent_Data() {} + type ClientMeta_Ethereum struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2936,7 +3015,7 @@ type ClientMeta_Ethereum struct { func (x *ClientMeta_Ethereum) Reset() { *x = ClientMeta_Ethereum{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[19] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2949,7 +3028,7 @@ func (x *ClientMeta_Ethereum) String() string { func (*ClientMeta_Ethereum) ProtoMessage() {} func (x *ClientMeta_Ethereum) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[19] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2962,7 +3041,7 @@ func (x *ClientMeta_Ethereum) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta_Ethereum.ProtoReflect.Descriptor instead. func (*ClientMeta_Ethereum) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 0} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 0} } func (x *ClientMeta_Ethereum) GetNetwork() *ClientMeta_Ethereum_Network { @@ -2998,7 +3077,7 @@ type ClientMeta_AdditionalEthV1AttestationSourceData struct { func (x *ClientMeta_AdditionalEthV1AttestationSourceData) Reset() { *x = ClientMeta_AdditionalEthV1AttestationSourceData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[21] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3011,7 +3090,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationSourceData) String() string { func (*ClientMeta_AdditionalEthV1AttestationSourceData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1AttestationSourceData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[21] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3024,7 +3103,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationSourceData) ProtoReflect() protore // Deprecated: Use ClientMeta_AdditionalEthV1AttestationSourceData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1AttestationSourceData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 2} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 2} } func (x *ClientMeta_AdditionalEthV1AttestationSourceData) GetEpoch() *Epoch { @@ -3046,7 +3125,7 @@ type ClientMeta_AdditionalEthV1AttestationSourceV2Data struct { func (x *ClientMeta_AdditionalEthV1AttestationSourceV2Data) Reset() { *x = ClientMeta_AdditionalEthV1AttestationSourceV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[22] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3059,7 +3138,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationSourceV2Data) String() string { func (*ClientMeta_AdditionalEthV1AttestationSourceV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1AttestationSourceV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[22] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3072,7 +3151,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationSourceV2Data) ProtoReflect() proto // Deprecated: Use ClientMeta_AdditionalEthV1AttestationSourceV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1AttestationSourceV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 3} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 3} } func (x *ClientMeta_AdditionalEthV1AttestationSourceV2Data) GetEpoch() *EpochV2 { @@ -3094,7 +3173,7 @@ type ClientMeta_AdditionalEthV1AttestationTargetData struct { func (x *ClientMeta_AdditionalEthV1AttestationTargetData) Reset() { *x = ClientMeta_AdditionalEthV1AttestationTargetData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[23] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3107,7 +3186,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationTargetData) String() string { func (*ClientMeta_AdditionalEthV1AttestationTargetData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1AttestationTargetData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[23] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3120,7 +3199,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationTargetData) ProtoReflect() protore // Deprecated: Use ClientMeta_AdditionalEthV1AttestationTargetData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1AttestationTargetData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 4} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 4} } func (x *ClientMeta_AdditionalEthV1AttestationTargetData) GetEpoch() *Epoch { @@ -3142,7 +3221,7 @@ type ClientMeta_AdditionalEthV1AttestationTargetV2Data struct { func (x *ClientMeta_AdditionalEthV1AttestationTargetV2Data) Reset() { *x = ClientMeta_AdditionalEthV1AttestationTargetV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[24] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3155,7 +3234,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationTargetV2Data) String() string { func (*ClientMeta_AdditionalEthV1AttestationTargetV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1AttestationTargetV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[24] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3168,7 +3247,7 @@ func (x *ClientMeta_AdditionalEthV1AttestationTargetV2Data) ProtoReflect() proto // Deprecated: Use ClientMeta_AdditionalEthV1AttestationTargetV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1AttestationTargetV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 5} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 5} } func (x *ClientMeta_AdditionalEthV1AttestationTargetV2Data) GetEpoch() *EpochV2 { @@ -3203,7 +3282,7 @@ type ClientMeta_AdditionalEthV1EventsAttestationData struct { func (x *ClientMeta_AdditionalEthV1EventsAttestationData) Reset() { *x = ClientMeta_AdditionalEthV1EventsAttestationData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[25] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3216,7 +3295,7 @@ func (x *ClientMeta_AdditionalEthV1EventsAttestationData) String() string { func (*ClientMeta_AdditionalEthV1EventsAttestationData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsAttestationData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[25] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3229,7 +3308,7 @@ func (x *ClientMeta_AdditionalEthV1EventsAttestationData) ProtoReflect() protore // Deprecated: Use ClientMeta_AdditionalEthV1EventsAttestationData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsAttestationData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 6} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 6} } func (x *ClientMeta_AdditionalEthV1EventsAttestationData) GetSource() *ClientMeta_AdditionalEthV1AttestationSourceData { @@ -3299,7 +3378,7 @@ type ClientMeta_AdditionalEthV1EventsAttestationV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsAttestationV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsAttestationV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[26] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3312,7 +3391,7 @@ func (x *ClientMeta_AdditionalEthV1EventsAttestationV2Data) String() string { func (*ClientMeta_AdditionalEthV1EventsAttestationV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsAttestationV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[26] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3325,7 +3404,7 @@ func (x *ClientMeta_AdditionalEthV1EventsAttestationV2Data) ProtoReflect() proto // Deprecated: Use ClientMeta_AdditionalEthV1EventsAttestationV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsAttestationV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 7} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 7} } func (x *ClientMeta_AdditionalEthV1EventsAttestationV2Data) GetSource() *ClientMeta_AdditionalEthV1AttestationSourceV2Data { @@ -3386,7 +3465,7 @@ type ClientMeta_AdditionalEthV1EventsHeadData struct { func (x *ClientMeta_AdditionalEthV1EventsHeadData) Reset() { *x = ClientMeta_AdditionalEthV1EventsHeadData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[27] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3399,7 +3478,7 @@ func (x *ClientMeta_AdditionalEthV1EventsHeadData) String() string { func (*ClientMeta_AdditionalEthV1EventsHeadData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsHeadData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[27] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3412,7 +3491,7 @@ func (x *ClientMeta_AdditionalEthV1EventsHeadData) ProtoReflect() protoreflect.M // Deprecated: Use ClientMeta_AdditionalEthV1EventsHeadData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsHeadData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 8} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 8} } func (x *ClientMeta_AdditionalEthV1EventsHeadData) GetEpoch() *Epoch { @@ -3452,7 +3531,7 @@ type ClientMeta_AdditionalEthV1EventsHeadV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsHeadV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsHeadV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[28] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3465,7 +3544,7 @@ func (x *ClientMeta_AdditionalEthV1EventsHeadV2Data) String() string { func (*ClientMeta_AdditionalEthV1EventsHeadV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsHeadV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[28] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3478,7 +3557,7 @@ func (x *ClientMeta_AdditionalEthV1EventsHeadV2Data) ProtoReflect() protoreflect // Deprecated: Use ClientMeta_AdditionalEthV1EventsHeadV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsHeadV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 9} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 9} } func (x *ClientMeta_AdditionalEthV1EventsHeadV2Data) GetEpoch() *EpochV2 { @@ -3518,7 +3597,7 @@ type ClientMeta_AdditionalEthV1EventsBlockData struct { func (x *ClientMeta_AdditionalEthV1EventsBlockData) Reset() { *x = ClientMeta_AdditionalEthV1EventsBlockData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[29] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3531,7 +3610,7 @@ func (x *ClientMeta_AdditionalEthV1EventsBlockData) String() string { func (*ClientMeta_AdditionalEthV1EventsBlockData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsBlockData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[29] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3544,7 +3623,7 @@ func (x *ClientMeta_AdditionalEthV1EventsBlockData) ProtoReflect() protoreflect. // Deprecated: Use ClientMeta_AdditionalEthV1EventsBlockData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsBlockData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 10} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 10} } func (x *ClientMeta_AdditionalEthV1EventsBlockData) GetEpoch() *Epoch { @@ -3584,7 +3663,7 @@ type ClientMeta_AdditionalEthV1EventsBlockV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsBlockV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsBlockV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[30] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3597,7 +3676,7 @@ func (x *ClientMeta_AdditionalEthV1EventsBlockV2Data) String() string { func (*ClientMeta_AdditionalEthV1EventsBlockV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsBlockV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[30] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3610,7 +3689,7 @@ func (x *ClientMeta_AdditionalEthV1EventsBlockV2Data) ProtoReflect() protoreflec // Deprecated: Use ClientMeta_AdditionalEthV1EventsBlockV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsBlockV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 11} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 11} } func (x *ClientMeta_AdditionalEthV1EventsBlockV2Data) GetEpoch() *EpochV2 { @@ -3646,7 +3725,7 @@ type ClientMeta_AdditionalEthV1EventsVoluntaryExitData struct { func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) Reset() { *x = ClientMeta_AdditionalEthV1EventsVoluntaryExitData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[31] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3659,7 +3738,7 @@ func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) String() string { func (*ClientMeta_AdditionalEthV1EventsVoluntaryExitData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[31] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3672,7 +3751,7 @@ func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) ProtoReflect() proto // Deprecated: Use ClientMeta_AdditionalEthV1EventsVoluntaryExitData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsVoluntaryExitData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 12} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 12} } func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitData) GetEpoch() *Epoch { @@ -3694,7 +3773,7 @@ type ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[32] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3707,7 +3786,7 @@ func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) String() string { func (*ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[32] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3720,7 +3799,7 @@ func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) ProtoReflect() pro // Deprecated: Use ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 13} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 13} } func (x *ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data) GetEpoch() *EpochV2 { @@ -3742,7 +3821,7 @@ type ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData struct { func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) Reset() { *x = ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[33] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3755,7 +3834,7 @@ func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) String() strin func (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[33] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3768,7 +3847,7 @@ func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) ProtoReflect() // Deprecated: Use ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 14} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 14} } func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData) GetEpoch() *Epoch { @@ -3790,7 +3869,7 @@ type ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[34] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3803,7 +3882,7 @@ func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) String() str func (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[34] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3816,7 +3895,7 @@ func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) ProtoReflect // Deprecated: Use ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 15} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 15} } func (x *ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data) GetEpoch() *EpochV2 { @@ -3843,7 +3922,7 @@ type ClientMeta_AdditionalEthV1EventsChainReorgData struct { func (x *ClientMeta_AdditionalEthV1EventsChainReorgData) Reset() { *x = ClientMeta_AdditionalEthV1EventsChainReorgData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[35] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[36] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3856,7 +3935,7 @@ func (x *ClientMeta_AdditionalEthV1EventsChainReorgData) String() string { func (*ClientMeta_AdditionalEthV1EventsChainReorgData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsChainReorgData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[35] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[36] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3869,7 +3948,7 @@ func (x *ClientMeta_AdditionalEthV1EventsChainReorgData) ProtoReflect() protoref // Deprecated: Use ClientMeta_AdditionalEthV1EventsChainReorgData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsChainReorgData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 16} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 16} } func (x *ClientMeta_AdditionalEthV1EventsChainReorgData) GetEpoch() *Epoch { @@ -3910,7 +3989,7 @@ type ClientMeta_AdditionalEthV1EventsChainReorgV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsChainReorgV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[36] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3923,7 +4002,7 @@ func (x *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) String() string { func (*ClientMeta_AdditionalEthV1EventsChainReorgV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[36] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3936,7 +4015,7 @@ func (x *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) ProtoReflect() protor // Deprecated: Use ClientMeta_AdditionalEthV1EventsChainReorgV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsChainReorgV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 17} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 17} } func (x *ClientMeta_AdditionalEthV1EventsChainReorgV2Data) GetEpoch() *EpochV2 { @@ -3977,7 +4056,7 @@ type ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData struct func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) Reset() { *x = ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[37] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3990,7 +4069,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) S func (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[37] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4003,7 +4082,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) P // Deprecated: Use ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 18} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 18} } func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData) GetEpoch() *Epoch { @@ -4044,7 +4123,7 @@ type ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data stru func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[38] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4057,7 +4136,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) func (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[38] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4070,7 +4149,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) // Deprecated: Use ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 19} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 19} } func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data) GetEpoch() *EpochV2 { @@ -4105,7 +4184,7 @@ type ClientMeta_AdditionalEthV1EventsContributionAndProofData struct { func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofData) Reset() { *x = ClientMeta_AdditionalEthV1EventsContributionAndProofData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[39] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4118,7 +4197,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofData) String() stri func (*ClientMeta_AdditionalEthV1EventsContributionAndProofData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[39] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4131,7 +4210,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofData) ProtoReflect( // Deprecated: Use ClientMeta_AdditionalEthV1EventsContributionAndProofData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsContributionAndProofData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 20} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 20} } func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofData) GetContribution() *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData { @@ -4152,7 +4231,7 @@ type ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data struct { func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) Reset() { *x = ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[40] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4165,7 +4244,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) String() st func (*ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[40] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4178,7 +4257,7 @@ func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) ProtoReflec // Deprecated: Use ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 21} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 21} } func (x *ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data) GetContribution() *ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data { @@ -4212,7 +4291,7 @@ type ClientMeta_ForkChoiceSnapshot struct { func (x *ClientMeta_ForkChoiceSnapshot) Reset() { *x = ClientMeta_ForkChoiceSnapshot{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[41] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[42] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4225,7 +4304,7 @@ func (x *ClientMeta_ForkChoiceSnapshot) String() string { func (*ClientMeta_ForkChoiceSnapshot) ProtoMessage() {} func (x *ClientMeta_ForkChoiceSnapshot) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[41] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[42] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4238,7 +4317,7 @@ func (x *ClientMeta_ForkChoiceSnapshot) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta_ForkChoiceSnapshot.ProtoReflect.Descriptor instead. func (*ClientMeta_ForkChoiceSnapshot) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 22} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 22} } func (x *ClientMeta_ForkChoiceSnapshot) GetRequestEpoch() *Epoch { @@ -4300,7 +4379,7 @@ type ClientMeta_ForkChoiceSnapshotV2 struct { func (x *ClientMeta_ForkChoiceSnapshotV2) Reset() { *x = ClientMeta_ForkChoiceSnapshotV2{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[42] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[43] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4313,7 +4392,7 @@ func (x *ClientMeta_ForkChoiceSnapshotV2) String() string { func (*ClientMeta_ForkChoiceSnapshotV2) ProtoMessage() {} func (x *ClientMeta_ForkChoiceSnapshotV2) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[42] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[43] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4326,7 +4405,7 @@ func (x *ClientMeta_ForkChoiceSnapshotV2) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta_ForkChoiceSnapshotV2.ProtoReflect.Descriptor instead. func (*ClientMeta_ForkChoiceSnapshotV2) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 23} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 23} } func (x *ClientMeta_ForkChoiceSnapshotV2) GetRequestEpoch() *EpochV2 { @@ -4375,7 +4454,7 @@ type ClientMeta_AdditionalEthV1DebugForkChoiceData struct { func (x *ClientMeta_AdditionalEthV1DebugForkChoiceData) Reset() { *x = ClientMeta_AdditionalEthV1DebugForkChoiceData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[43] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4388,7 +4467,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceData) String() string { func (*ClientMeta_AdditionalEthV1DebugForkChoiceData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1DebugForkChoiceData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[43] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4401,7 +4480,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceData) ProtoReflect() protorefl // Deprecated: Use ClientMeta_AdditionalEthV1DebugForkChoiceData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1DebugForkChoiceData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 24} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 24} } func (x *ClientMeta_AdditionalEthV1DebugForkChoiceData) GetSnapshot() *ClientMeta_ForkChoiceSnapshot { @@ -4422,7 +4501,7 @@ type ClientMeta_AdditionalEthV1DebugForkChoiceV2Data struct { func (x *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) Reset() { *x = ClientMeta_AdditionalEthV1DebugForkChoiceV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[44] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4435,7 +4514,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) String() string { func (*ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[44] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4448,7 +4527,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) ProtoReflect() protore // Deprecated: Use ClientMeta_AdditionalEthV1DebugForkChoiceV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 25} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 25} } func (x *ClientMeta_AdditionalEthV1DebugForkChoiceV2Data) GetSnapshot() *ClientMeta_ForkChoiceSnapshotV2 { @@ -4470,7 +4549,7 @@ type ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData struct { func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) Reset() { *x = ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[45] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4483,7 +4562,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) String() string { func (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[45] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4496,7 +4575,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) ProtoReflect() prot // Deprecated: Use ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 26} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 26} } func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData) GetBefore() *ClientMeta_ForkChoiceSnapshot { @@ -4525,7 +4604,7 @@ type ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data struct { func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) Reset() { *x = ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[46] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4538,7 +4617,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) String() string { func (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[46] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4551,7 +4630,7 @@ func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) ProtoReflect() pr // Deprecated: Use ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 27} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 27} } func (x *ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data) GetBefore() *ClientMeta_ForkChoiceSnapshotV2 { @@ -4582,7 +4661,7 @@ type ClientMeta_AdditionalEthV1BeaconCommitteeData struct { func (x *ClientMeta_AdditionalEthV1BeaconCommitteeData) Reset() { *x = ClientMeta_AdditionalEthV1BeaconCommitteeData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[47] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4595,7 +4674,7 @@ func (x *ClientMeta_AdditionalEthV1BeaconCommitteeData) String() string { func (*ClientMeta_AdditionalEthV1BeaconCommitteeData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1BeaconCommitteeData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[47] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4608,7 +4687,7 @@ func (x *ClientMeta_AdditionalEthV1BeaconCommitteeData) ProtoReflect() protorefl // Deprecated: Use ClientMeta_AdditionalEthV1BeaconCommitteeData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1BeaconCommitteeData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 28} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 28} } func (x *ClientMeta_AdditionalEthV1BeaconCommitteeData) GetEpoch() *EpochV2 { @@ -4653,7 +4732,7 @@ type ClientMeta_AdditionalMempoolTransactionData struct { func (x *ClientMeta_AdditionalMempoolTransactionData) Reset() { *x = ClientMeta_AdditionalMempoolTransactionData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[48] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4666,7 +4745,7 @@ func (x *ClientMeta_AdditionalMempoolTransactionData) String() string { func (*ClientMeta_AdditionalMempoolTransactionData) ProtoMessage() {} func (x *ClientMeta_AdditionalMempoolTransactionData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[48] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4679,7 +4758,7 @@ func (x *ClientMeta_AdditionalMempoolTransactionData) ProtoReflect() protoreflec // Deprecated: Use ClientMeta_AdditionalMempoolTransactionData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalMempoolTransactionData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 29} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 29} } func (x *ClientMeta_AdditionalMempoolTransactionData) GetHash() string { @@ -4789,7 +4868,7 @@ type ClientMeta_AdditionalMempoolTransactionV2Data struct { func (x *ClientMeta_AdditionalMempoolTransactionV2Data) Reset() { *x = ClientMeta_AdditionalMempoolTransactionV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[49] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4802,7 +4881,7 @@ func (x *ClientMeta_AdditionalMempoolTransactionV2Data) String() string { func (*ClientMeta_AdditionalMempoolTransactionV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalMempoolTransactionV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[49] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4815,7 +4894,7 @@ func (x *ClientMeta_AdditionalMempoolTransactionV2Data) ProtoReflect() protorefl // Deprecated: Use ClientMeta_AdditionalMempoolTransactionV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalMempoolTransactionV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 30} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 30} } func (x *ClientMeta_AdditionalMempoolTransactionV2Data) GetHash() string { @@ -4959,7 +5038,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[50] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4972,7 +5051,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockData) String() string { func (*ClientMeta_AdditionalEthV2BeaconBlockData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[50] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4985,7 +5064,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockData) ProtoReflect() protoreflect. // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 31} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 31} } func (x *ClientMeta_AdditionalEthV2BeaconBlockData) GetEpoch() *Epoch { @@ -5062,7 +5141,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockV2Data struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockV2Data) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockV2Data{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[51] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5075,7 +5154,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockV2Data) String() string { func (*ClientMeta_AdditionalEthV2BeaconBlockV2Data) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockV2Data) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[51] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5088,7 +5167,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockV2Data) ProtoReflect() protoreflec // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockV2Data.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockV2Data) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 32} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 32} } func (x *ClientMeta_AdditionalEthV2BeaconBlockV2Data) GetEpoch() *EpochV2 { @@ -5174,7 +5253,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[52] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5187,7 +5266,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) String() str func (*ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[52] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5200,7 +5279,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) ProtoReflect // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 33} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 33} } func (x *ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData) GetBlock() *BlockIdentifier { @@ -5223,7 +5302,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[53] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5236,7 +5315,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) String() str func (*ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[53] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5249,7 +5328,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) ProtoReflect // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 34} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 34} } func (x *ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData) GetBlock() *BlockIdentifier { @@ -5272,7 +5351,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[54] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5285,7 +5364,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) String() string func (*ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[54] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5298,7 +5377,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) ProtoReflect() // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 35} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 35} } func (x *ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData) GetBlock() *BlockIdentifier { @@ -5321,7 +5400,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockDepositData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockDepositData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockDepositData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[55] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5334,7 +5413,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockDepositData) String() string { func (*ClientMeta_AdditionalEthV2BeaconBlockDepositData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockDepositData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[55] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5347,7 +5426,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockDepositData) ProtoReflect() protor // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockDepositData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockDepositData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 36} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 36} } func (x *ClientMeta_AdditionalEthV2BeaconBlockDepositData) GetBlock() *BlockIdentifier { @@ -5370,7 +5449,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[56] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5383,7 +5462,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) String() func (*ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[56] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5396,7 +5475,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) ProtoRef // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 37} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 37} } func (x *ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData) GetBlock() *BlockIdentifier { @@ -5429,7 +5508,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[57] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5442,7 +5521,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) String() func (*ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[57] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5455,7 +5534,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) ProtoRef // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 38} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 38} } func (x *ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData) GetBlock() *BlockIdentifier { @@ -5513,7 +5592,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[58] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5526,7 +5605,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) String() string { func (*ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[58] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5539,7 +5618,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) ProtoReflect() pro // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 39} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 39} } func (x *ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData) GetBlock() *BlockIdentifier { @@ -5563,7 +5642,7 @@ type ClientMeta_AdditionalBlockprintBlockClassificationData struct { func (x *ClientMeta_AdditionalBlockprintBlockClassificationData) Reset() { *x = ClientMeta_AdditionalBlockprintBlockClassificationData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[59] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5576,7 +5655,7 @@ func (x *ClientMeta_AdditionalBlockprintBlockClassificationData) String() string func (*ClientMeta_AdditionalBlockprintBlockClassificationData) ProtoMessage() {} func (x *ClientMeta_AdditionalBlockprintBlockClassificationData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[59] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[60] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5589,7 +5668,7 @@ func (x *ClientMeta_AdditionalBlockprintBlockClassificationData) ProtoReflect() // Deprecated: Use ClientMeta_AdditionalBlockprintBlockClassificationData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalBlockprintBlockClassificationData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 40} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 40} } func (x *ClientMeta_AdditionalBlockprintBlockClassificationData) GetEpoch() *EpochV2 { @@ -5625,7 +5704,7 @@ type ClientMeta_AttestationDataSnapshot struct { func (x *ClientMeta_AttestationDataSnapshot) Reset() { *x = ClientMeta_AttestationDataSnapshot{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[60] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5638,7 +5717,7 @@ func (x *ClientMeta_AttestationDataSnapshot) String() string { func (*ClientMeta_AttestationDataSnapshot) ProtoMessage() {} func (x *ClientMeta_AttestationDataSnapshot) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[60] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[61] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5651,7 +5730,7 @@ func (x *ClientMeta_AttestationDataSnapshot) ProtoReflect() protoreflect.Message // Deprecated: Use ClientMeta_AttestationDataSnapshot.ProtoReflect.Descriptor instead. func (*ClientMeta_AttestationDataSnapshot) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 41} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 41} } func (x *ClientMeta_AttestationDataSnapshot) GetRequestedAtSlotStartDiffMs() *wrapperspb.UInt64Value { @@ -5696,7 +5775,7 @@ type ClientMeta_AdditionalEthV1ValidatorAttestationDataData struct { func (x *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) Reset() { *x = ClientMeta_AdditionalEthV1ValidatorAttestationDataData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[61] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5709,7 +5788,7 @@ func (x *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) String() string func (*ClientMeta_AdditionalEthV1ValidatorAttestationDataData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[61] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[62] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5722,7 +5801,7 @@ func (x *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) ProtoReflect() // Deprecated: Use ClientMeta_AdditionalEthV1ValidatorAttestationDataData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1ValidatorAttestationDataData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 42} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 42} } func (x *ClientMeta_AdditionalEthV1ValidatorAttestationDataData) GetSource() *ClientMeta_AdditionalEthV1AttestationSourceV2Data { @@ -5777,7 +5856,7 @@ type ClientMeta_AdditionalEthV1EventsBlobSidecarData struct { func (x *ClientMeta_AdditionalEthV1EventsBlobSidecarData) Reset() { *x = ClientMeta_AdditionalEthV1EventsBlobSidecarData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[62] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5790,7 +5869,7 @@ func (x *ClientMeta_AdditionalEthV1EventsBlobSidecarData) String() string { func (*ClientMeta_AdditionalEthV1EventsBlobSidecarData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1EventsBlobSidecarData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[62] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[63] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5803,7 +5882,7 @@ func (x *ClientMeta_AdditionalEthV1EventsBlobSidecarData) ProtoReflect() protore // Deprecated: Use ClientMeta_AdditionalEthV1EventsBlobSidecarData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1EventsBlobSidecarData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 43} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 43} } func (x *ClientMeta_AdditionalEthV1EventsBlobSidecarData) GetEpoch() *EpochV2 { @@ -5847,7 +5926,7 @@ type ClientMeta_AdditionalEthV1BeaconBlobSidecarData struct { func (x *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) Reset() { *x = ClientMeta_AdditionalEthV1BeaconBlobSidecarData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[63] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5860,7 +5939,7 @@ func (x *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) String() string { func (*ClientMeta_AdditionalEthV1BeaconBlobSidecarData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[63] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[64] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5873,7 +5952,7 @@ func (x *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) ProtoReflect() protore // Deprecated: Use ClientMeta_AdditionalEthV1BeaconBlobSidecarData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1BeaconBlobSidecarData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 44} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 44} } func (x *ClientMeta_AdditionalEthV1BeaconBlobSidecarData) GetEpoch() *EpochV2 { @@ -5942,7 +6021,7 @@ type ClientMeta_AdditionalBeaconP2PAttestationData struct { func (x *ClientMeta_AdditionalBeaconP2PAttestationData) Reset() { *x = ClientMeta_AdditionalBeaconP2PAttestationData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[64] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5955,7 +6034,7 @@ func (x *ClientMeta_AdditionalBeaconP2PAttestationData) String() string { func (*ClientMeta_AdditionalBeaconP2PAttestationData) ProtoMessage() {} func (x *ClientMeta_AdditionalBeaconP2PAttestationData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[64] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[65] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5968,7 +6047,7 @@ func (x *ClientMeta_AdditionalBeaconP2PAttestationData) ProtoReflect() protorefl // Deprecated: Use ClientMeta_AdditionalBeaconP2PAttestationData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalBeaconP2PAttestationData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 45} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 45} } func (x *ClientMeta_AdditionalBeaconP2PAttestationData) GetSource() *ClientMeta_AdditionalEthV1AttestationSourceV2Data { @@ -6052,7 +6131,7 @@ type ClientMeta_AdditionalEthV1ProposerDutyData struct { func (x *ClientMeta_AdditionalEthV1ProposerDutyData) Reset() { *x = ClientMeta_AdditionalEthV1ProposerDutyData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[65] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6065,7 +6144,7 @@ func (x *ClientMeta_AdditionalEthV1ProposerDutyData) String() string { func (*ClientMeta_AdditionalEthV1ProposerDutyData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV1ProposerDutyData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[65] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[66] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6078,7 +6157,7 @@ func (x *ClientMeta_AdditionalEthV1ProposerDutyData) ProtoReflect() protoreflect // Deprecated: Use ClientMeta_AdditionalEthV1ProposerDutyData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV1ProposerDutyData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 46} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 46} } func (x *ClientMeta_AdditionalEthV1ProposerDutyData) GetEpoch() *EpochV2 { @@ -6126,7 +6205,7 @@ type ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData struct { func (x *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Reset() { *x = ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[66] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6139,7 +6218,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) String( func (*ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) ProtoMessage() {} func (x *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[66] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[67] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6152,7 +6231,7 @@ func (x *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) ProtoRe // Deprecated: Use ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 47} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 47} } func (x *ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData) GetBlock() *BlockIdentifier { @@ -6209,7 +6288,7 @@ type ClientMeta_AdditionalLibP2PTraceAddPeerData struct { func (x *ClientMeta_AdditionalLibP2PTraceAddPeerData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceAddPeerData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[67] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6222,7 +6301,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceAddPeerData) String() string { func (*ClientMeta_AdditionalLibP2PTraceAddPeerData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceAddPeerData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[67] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[68] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6235,7 +6314,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceAddPeerData) ProtoReflect() protoreflec // Deprecated: Use ClientMeta_AdditionalLibP2PTraceAddPeerData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceAddPeerData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 48} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 48} } func (x *ClientMeta_AdditionalLibP2PTraceAddPeerData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6257,7 +6336,7 @@ type ClientMeta_AdditionalLibP2PTraceRemovePeerData struct { func (x *ClientMeta_AdditionalLibP2PTraceRemovePeerData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceRemovePeerData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[68] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6270,7 +6349,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceRemovePeerData) String() string { func (*ClientMeta_AdditionalLibP2PTraceRemovePeerData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceRemovePeerData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[68] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[69] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6283,7 +6362,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceRemovePeerData) ProtoReflect() protoref // Deprecated: Use ClientMeta_AdditionalLibP2PTraceRemovePeerData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceRemovePeerData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 49} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 49} } func (x *ClientMeta_AdditionalLibP2PTraceRemovePeerData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6305,7 +6384,7 @@ type ClientMeta_AdditionalLibP2PTraceRecvRPCData struct { func (x *ClientMeta_AdditionalLibP2PTraceRecvRPCData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceRecvRPCData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[69] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6318,7 +6397,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceRecvRPCData) String() string { func (*ClientMeta_AdditionalLibP2PTraceRecvRPCData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceRecvRPCData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[69] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[70] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6331,7 +6410,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceRecvRPCData) ProtoReflect() protoreflec // Deprecated: Use ClientMeta_AdditionalLibP2PTraceRecvRPCData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceRecvRPCData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 50} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 50} } func (x *ClientMeta_AdditionalLibP2PTraceRecvRPCData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6353,7 +6432,7 @@ type ClientMeta_AdditionalLibP2PTraceSendRPCData struct { func (x *ClientMeta_AdditionalLibP2PTraceSendRPCData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceSendRPCData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[70] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6366,7 +6445,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceSendRPCData) String() string { func (*ClientMeta_AdditionalLibP2PTraceSendRPCData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceSendRPCData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[70] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[71] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6379,7 +6458,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceSendRPCData) ProtoReflect() protoreflec // Deprecated: Use ClientMeta_AdditionalLibP2PTraceSendRPCData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceSendRPCData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 51} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 51} } func (x *ClientMeta_AdditionalLibP2PTraceSendRPCData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6401,7 +6480,7 @@ type ClientMeta_AdditionalLibP2PTraceJoinData struct { func (x *ClientMeta_AdditionalLibP2PTraceJoinData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceJoinData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[71] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6414,7 +6493,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceJoinData) String() string { func (*ClientMeta_AdditionalLibP2PTraceJoinData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceJoinData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[71] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[72] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6427,7 +6506,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceJoinData) ProtoReflect() protoreflect.M // Deprecated: Use ClientMeta_AdditionalLibP2PTraceJoinData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceJoinData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 52} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 52} } func (x *ClientMeta_AdditionalLibP2PTraceJoinData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6449,7 +6528,7 @@ type ClientMeta_AdditionalLibP2PTraceConnectedData struct { func (x *ClientMeta_AdditionalLibP2PTraceConnectedData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceConnectedData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[72] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6462,7 +6541,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceConnectedData) String() string { func (*ClientMeta_AdditionalLibP2PTraceConnectedData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceConnectedData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[72] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[73] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6475,7 +6554,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceConnectedData) ProtoReflect() protorefl // Deprecated: Use ClientMeta_AdditionalLibP2PTraceConnectedData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceConnectedData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 53} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 53} } func (x *ClientMeta_AdditionalLibP2PTraceConnectedData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6497,7 +6576,7 @@ type ClientMeta_AdditionalLibP2PTraceDisconnectedData struct { func (x *ClientMeta_AdditionalLibP2PTraceDisconnectedData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceDisconnectedData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[73] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6510,7 +6589,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceDisconnectedData) String() string { func (*ClientMeta_AdditionalLibP2PTraceDisconnectedData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceDisconnectedData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[73] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[74] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6523,7 +6602,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceDisconnectedData) ProtoReflect() protor // Deprecated: Use ClientMeta_AdditionalLibP2PTraceDisconnectedData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceDisconnectedData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 54} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 54} } func (x *ClientMeta_AdditionalLibP2PTraceDisconnectedData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6545,7 +6624,7 @@ type ClientMeta_AdditionalLibP2PTraceHandleMetadataData struct { func (x *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceHandleMetadataData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[74] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6558,7 +6637,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) String() string { func (*ClientMeta_AdditionalLibP2PTraceHandleMetadataData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[74] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[75] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6571,7 +6650,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) ProtoReflect() prot // Deprecated: Use ClientMeta_AdditionalLibP2PTraceHandleMetadataData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceHandleMetadataData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 55} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 55} } func (x *ClientMeta_AdditionalLibP2PTraceHandleMetadataData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6593,7 +6672,7 @@ type ClientMeta_AdditionalLibP2PTraceHandleStatusData struct { func (x *ClientMeta_AdditionalLibP2PTraceHandleStatusData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceHandleStatusData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[75] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[76] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6606,7 +6685,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceHandleStatusData) String() string { func (*ClientMeta_AdditionalLibP2PTraceHandleStatusData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceHandleStatusData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[75] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[76] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6619,7 +6698,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceHandleStatusData) ProtoReflect() protor // Deprecated: Use ClientMeta_AdditionalLibP2PTraceHandleStatusData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceHandleStatusData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 56} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 56} } func (x *ClientMeta_AdditionalLibP2PTraceHandleStatusData) GetMetadata() *libp2p.TraceEventMetadata { @@ -6658,7 +6737,7 @@ type ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData struct { func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[76] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[77] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6671,7 +6750,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) String() stri func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[76] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[77] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6684,7 +6763,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) ProtoReflect( // Deprecated: Use ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 57} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 57} } func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData) GetEpoch() *EpochV2 { @@ -6762,7 +6841,7 @@ type ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData struct func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[77] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[78] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6775,7 +6854,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) S func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[77] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[78] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6788,7 +6867,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) P // Deprecated: Use ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 58} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 58} } func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData) GetEpoch() *EpochV2 { @@ -6810,7 +6889,7 @@ type ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData struct func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[78] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[79] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6823,7 +6902,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) S func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[78] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[79] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6836,7 +6915,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) P // Deprecated: Use ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 59} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 59} } func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData) GetEpoch() *EpochV2 { @@ -6883,7 +6962,7 @@ type ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData struct { func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[79] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[80] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6896,7 +6975,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) String( func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[79] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[80] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6909,7 +6988,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) ProtoRe // Deprecated: Use ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 60} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 60} } func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData) GetSource() *ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData { @@ -7024,7 +7103,7 @@ type ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData struct { func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) Reset() { *x = ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[80] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[81] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7037,7 +7116,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) String() stri func (*ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) ProtoMessage() {} func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[80] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[81] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7050,7 +7129,7 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) ProtoReflect( // Deprecated: Use ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData.ProtoReflect.Descriptor instead. func (*ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 61} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 61} } func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) GetEpoch() *EpochV2 { @@ -7116,6 +7195,54 @@ func (x *ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData) GetMessageId( return nil } +// AdditionalEthV1ValidatorsData contains additional data about the eth v1 validators. +type ClientMeta_AdditionalEthV1ValidatorsData struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Epoch *EpochV2 `protobuf:"bytes,1,opt,name=epoch,proto3" json:"epoch,omitempty"` +} + +func (x *ClientMeta_AdditionalEthV1ValidatorsData) Reset() { + *x = ClientMeta_AdditionalEthV1ValidatorsData{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[82] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ClientMeta_AdditionalEthV1ValidatorsData) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ClientMeta_AdditionalEthV1ValidatorsData) ProtoMessage() {} + +func (x *ClientMeta_AdditionalEthV1ValidatorsData) ProtoReflect() protoreflect.Message { + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[82] + 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 ClientMeta_AdditionalEthV1ValidatorsData.ProtoReflect.Descriptor instead. +func (*ClientMeta_AdditionalEthV1ValidatorsData) Descriptor() ([]byte, []int) { + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 62} +} + +func (x *ClientMeta_AdditionalEthV1ValidatorsData) GetEpoch() *EpochV2 { + if x != nil { + return x.Epoch + } + return nil +} + type ClientMeta_Ethereum_Network struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7130,7 +7257,7 @@ type ClientMeta_Ethereum_Network struct { func (x *ClientMeta_Ethereum_Network) Reset() { *x = ClientMeta_Ethereum_Network{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[81] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7143,7 +7270,7 @@ func (x *ClientMeta_Ethereum_Network) String() string { func (*ClientMeta_Ethereum_Network) ProtoMessage() {} func (x *ClientMeta_Ethereum_Network) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[81] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[83] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7156,7 +7283,7 @@ func (x *ClientMeta_Ethereum_Network) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta_Ethereum_Network.ProtoReflect.Descriptor instead. func (*ClientMeta_Ethereum_Network) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 0, 0} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 0, 0} } func (x *ClientMeta_Ethereum_Network) GetName() string { @@ -7185,7 +7312,7 @@ type ClientMeta_Ethereum_Execution struct { func (x *ClientMeta_Ethereum_Execution) Reset() { *x = ClientMeta_Ethereum_Execution{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[82] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7198,7 +7325,7 @@ func (x *ClientMeta_Ethereum_Execution) String() string { func (*ClientMeta_Ethereum_Execution) ProtoMessage() {} func (x *ClientMeta_Ethereum_Execution) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[82] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[84] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7211,7 +7338,7 @@ func (x *ClientMeta_Ethereum_Execution) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta_Ethereum_Execution.ProtoReflect.Descriptor instead. func (*ClientMeta_Ethereum_Execution) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 0, 1} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 0, 1} } func (x *ClientMeta_Ethereum_Execution) GetForkId() *ForkID { @@ -7235,7 +7362,7 @@ type ClientMeta_Ethereum_Consensus struct { func (x *ClientMeta_Ethereum_Consensus) Reset() { *x = ClientMeta_Ethereum_Consensus{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[83] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[85] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7248,7 +7375,7 @@ func (x *ClientMeta_Ethereum_Consensus) String() string { func (*ClientMeta_Ethereum_Consensus) ProtoMessage() {} func (x *ClientMeta_Ethereum_Consensus) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[83] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[85] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7261,7 +7388,7 @@ func (x *ClientMeta_Ethereum_Consensus) ProtoReflect() protoreflect.Message { // Deprecated: Use ClientMeta_Ethereum_Consensus.ProtoReflect.Descriptor instead. func (*ClientMeta_Ethereum_Consensus) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{14, 0, 2} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 0, 2} } func (x *ClientMeta_Ethereum_Consensus) GetImplementation() string { @@ -7290,7 +7417,7 @@ type ServerMeta_Event struct { func (x *ServerMeta_Event) Reset() { *x = ServerMeta_Event{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[84] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[86] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7303,7 +7430,7 @@ func (x *ServerMeta_Event) String() string { func (*ServerMeta_Event) ProtoMessage() {} func (x *ServerMeta_Event) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[84] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[86] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7316,7 +7443,7 @@ func (x *ServerMeta_Event) ProtoReflect() protoreflect.Message { // Deprecated: Use ServerMeta_Event.ProtoReflect.Descriptor instead. func (*ServerMeta_Event) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 0} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 0} } func (x *ServerMeta_Event) GetReceivedDateTime() *timestamppb.Timestamp { @@ -7358,7 +7485,7 @@ type ServerMeta_Geo struct { func (x *ServerMeta_Geo) Reset() { *x = ServerMeta_Geo{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[85] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[87] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7371,7 +7498,7 @@ func (x *ServerMeta_Geo) String() string { func (*ServerMeta_Geo) ProtoMessage() {} func (x *ServerMeta_Geo) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[85] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[87] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7384,7 +7511,7 @@ func (x *ServerMeta_Geo) ProtoReflect() protoreflect.Message { // Deprecated: Use ServerMeta_Geo.ProtoReflect.Descriptor instead. func (*ServerMeta_Geo) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 1} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 1} } func (x *ServerMeta_Geo) GetCity() string { @@ -7458,7 +7585,7 @@ type ServerMeta_Client struct { func (x *ServerMeta_Client) Reset() { *x = ServerMeta_Client{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[86] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[88] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7471,7 +7598,7 @@ func (x *ServerMeta_Client) String() string { func (*ServerMeta_Client) ProtoMessage() {} func (x *ServerMeta_Client) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[86] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[88] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7484,7 +7611,7 @@ func (x *ServerMeta_Client) ProtoReflect() protoreflect.Message { // Deprecated: Use ServerMeta_Client.ProtoReflect.Descriptor instead. func (*ServerMeta_Client) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 2} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 2} } func (x *ServerMeta_Client) GetIP() string { @@ -7513,7 +7640,7 @@ type ServerMeta_Peer struct { func (x *ServerMeta_Peer) Reset() { *x = ServerMeta_Peer{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[87] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[89] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7526,7 +7653,7 @@ func (x *ServerMeta_Peer) String() string { func (*ServerMeta_Peer) ProtoMessage() {} func (x *ServerMeta_Peer) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[87] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[89] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7539,7 +7666,7 @@ func (x *ServerMeta_Peer) ProtoReflect() protoreflect.Message { // Deprecated: Use ServerMeta_Peer.ProtoReflect.Descriptor instead. func (*ServerMeta_Peer) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 3} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 3} } func (x *ServerMeta_Peer) GetGeo() *ServerMeta_Geo { @@ -7560,7 +7687,7 @@ type ServerMeta_AdditionalBeaconP2PAttestationData struct { func (x *ServerMeta_AdditionalBeaconP2PAttestationData) Reset() { *x = ServerMeta_AdditionalBeaconP2PAttestationData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[88] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[90] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7573,7 +7700,7 @@ func (x *ServerMeta_AdditionalBeaconP2PAttestationData) String() string { func (*ServerMeta_AdditionalBeaconP2PAttestationData) ProtoMessage() {} func (x *ServerMeta_AdditionalBeaconP2PAttestationData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[88] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[90] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7586,7 +7713,7 @@ func (x *ServerMeta_AdditionalBeaconP2PAttestationData) ProtoReflect() protorefl // Deprecated: Use ServerMeta_AdditionalBeaconP2PAttestationData.ProtoReflect.Descriptor instead. func (*ServerMeta_AdditionalBeaconP2PAttestationData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 4} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 4} } func (x *ServerMeta_AdditionalBeaconP2PAttestationData) GetPeer() *ServerMeta_Peer { @@ -7607,7 +7734,7 @@ type ServerMeta_AdditionalLibp2PTraceConnectedData struct { func (x *ServerMeta_AdditionalLibp2PTraceConnectedData) Reset() { *x = ServerMeta_AdditionalLibp2PTraceConnectedData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[89] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[91] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7620,7 +7747,7 @@ func (x *ServerMeta_AdditionalLibp2PTraceConnectedData) String() string { func (*ServerMeta_AdditionalLibp2PTraceConnectedData) ProtoMessage() {} func (x *ServerMeta_AdditionalLibp2PTraceConnectedData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[89] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[91] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7633,7 +7760,7 @@ func (x *ServerMeta_AdditionalLibp2PTraceConnectedData) ProtoReflect() protorefl // Deprecated: Use ServerMeta_AdditionalLibp2PTraceConnectedData.ProtoReflect.Descriptor instead. func (*ServerMeta_AdditionalLibp2PTraceConnectedData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 5} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 5} } func (x *ServerMeta_AdditionalLibp2PTraceConnectedData) GetPeer() *ServerMeta_Peer { @@ -7654,7 +7781,7 @@ type ServerMeta_AdditionalLibp2PTraceDisconnectedData struct { func (x *ServerMeta_AdditionalLibp2PTraceDisconnectedData) Reset() { *x = ServerMeta_AdditionalLibp2PTraceDisconnectedData{} if protoimpl.UnsafeEnabled { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[90] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[92] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7667,7 +7794,7 @@ func (x *ServerMeta_AdditionalLibp2PTraceDisconnectedData) String() string { func (*ServerMeta_AdditionalLibp2PTraceDisconnectedData) ProtoMessage() {} func (x *ServerMeta_AdditionalLibp2PTraceDisconnectedData) ProtoReflect() protoreflect.Message { - mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[90] + mi := &file_pkg_proto_xatu_event_ingester_proto_msgTypes[92] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7680,7 +7807,7 @@ func (x *ServerMeta_AdditionalLibp2PTraceDisconnectedData) ProtoReflect() protor // Deprecated: Use ServerMeta_AdditionalLibp2PTraceDisconnectedData.ProtoReflect.Descriptor instead. func (*ServerMeta_AdditionalLibp2PTraceDisconnectedData) Descriptor() ([]byte, []int) { - return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{15, 6} + return file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP(), []int{16, 6} } func (x *ServerMeta_AdditionalLibp2PTraceDisconnectedData) GetPeer() *ServerMeta_Peer { @@ -7721,640 +7848,736 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 0x6f, 0x63, 0x6b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x27, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x1a, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, - 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6c, - 0x69, 0x62, 0x70, 0x32, 0x70, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x1a, 0x1c, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6c, 0x69, 0x62, 0x70, - 0x32, 0x70, 0x2f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x24, - 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, - 0x2f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x2f, 0x65, 0x74, 0x68, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x43, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, 0x06, 0x65, - 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x44, 0x65, 0x63, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x65, 0x0a, 0x05, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, + 0x74, 0x6f, 0x1a, 0x20, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x74, + 0x68, 0x2f, 0x76, 0x31, 0x2f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x2f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x1a, 0x1c, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6c, 0x69, + 0x62, 0x70, 0x32, 0x70, 0x2f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x1a, 0x24, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x2f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x2f, 0x65, 0x74, 0x68, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x43, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2c, 0x0a, + 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x44, 0x65, 0x63, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x16, 0x0a, 0x14, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x65, 0x0a, 0x05, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x16, 0x0a, 0x06, + 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, + 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 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, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x22, 0x85, 0x01, 0x0a, 0x07, 0x45, + 0x70, 0x6f, 0x63, 0x68, 0x56, 0x32, 0x12, 0x34, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 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, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x0f, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 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, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x22, 0x64, 0x0a, 0x04, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 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, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, - 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x22, 0x85, 0x01, 0x0a, 0x07, 0x45, 0x70, 0x6f, - 0x63, 0x68, 0x56, 0x32, 0x12, 0x34, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, + 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x22, 0x84, 0x01, 0x0a, 0x06, 0x53, 0x6c, 0x6f, + 0x74, 0x56, 0x32, 0x12, 0x34, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 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, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x0f, 0x73, 0x74, 0x61, + 0x72, 0x74, 0x5f, 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, 0x0f, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x22, + 0x30, 0x0a, 0x06, 0x46, 0x6f, 0x72, 0x6b, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, + 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, + 0x04, 0x6e, 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x65, 0x78, + 0x74, 0x22, 0x37, 0x0a, 0x0b, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x28, 0x0a, 0x0f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, + 0x69, 0x66, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x22, 0x57, 0x0a, 0x0d, 0x50, 0x72, + 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x12, 0x46, 0x0a, 0x0f, 0x73, + 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, 0x66, 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, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x0f, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x5f, 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, - 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x22, 0x64, 0x0a, 0x04, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, - 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, - 0x12, 0x44, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 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, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x61, 0x74, - 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x22, 0x84, 0x01, 0x0a, 0x06, 0x53, 0x6c, 0x6f, 0x74, 0x56, - 0x32, 0x12, 0x34, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 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, - 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x44, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x72, 0x74, - 0x5f, 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, 0x0f, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x5f, 0x64, 0x61, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x22, 0x30, 0x0a, - 0x06, 0x46, 0x6f, 0x72, 0x6b, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x6e, - 0x65, 0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x65, 0x78, 0x74, 0x22, - 0x37, 0x0a, 0x0b, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, - 0x0a, 0x0f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, - 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, 0x66, 0x22, 0x57, 0x0a, 0x0d, 0x50, 0x72, 0x6f, 0x70, - 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x12, 0x46, 0x0a, 0x0f, 0x73, 0x6c, 0x6f, - 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, 0x66, 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, 0x0f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, 0x69, 0x66, - 0x66, 0x22, 0x54, 0x0a, 0x12, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, - 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x28, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, - 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x22, 0x92, 0x01, 0x0a, 0x14, 0x41, 0x74, 0x74, 0x65, - 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x56, 0x32, - 0x12, 0x46, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x75, 0x65, 0x52, 0x0f, 0x73, 0x6c, 0x6f, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x64, + 0x69, 0x66, 0x66, 0x22, 0x54, 0x0a, 0x12, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, + 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x28, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, + 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x22, 0x92, 0x01, 0x0a, 0x14, 0x41, 0x74, + 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, + 0x56, 0x32, 0x12, 0x46, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x5f, + 0x69, 0x6e, 0x64, 0x65, 0x78, 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, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, + 0x74, 0x74, 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x32, 0x0a, 0x05, 0x69, 0x6e, + 0x64, 0x65, 0x78, 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, 0x0f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, - 0x65, 0x65, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x32, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, - 0x78, 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, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xaa, 0x01, 0x0a, - 0x14, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, - 0x52, 0x65, 0x6f, 0x72, 0x67, 0x12, 0x2f, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x18, - 0x01, 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, 0x52, 0x06, - 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x18, - 0x02, 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, 0x52, 0x05, - 0x61, 0x66, 0x74, 0x65, 0x72, 0x12, 0x32, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18, 0x03, - 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, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0xb2, 0x01, 0x0a, 0x16, 0x44, 0x65, - 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x6f, - 0x72, 0x67, 0x56, 0x32, 0x12, 0x31, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x18, 0x01, - 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, 0x52, - 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x2f, 0x0a, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, + 0x36, 0x34, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x22, 0xaa, + 0x01, 0x0a, 0x14, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, + 0x63, 0x65, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x12, 0x2f, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, + 0x65, 0x18, 0x01, 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, + 0x52, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x2d, 0x0a, 0x05, 0x61, 0x66, 0x74, 0x65, + 0x72, 0x18, 0x02, 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, + 0x52, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x12, 0x32, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x18, 0x03, 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, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0xb2, 0x01, 0x0a, 0x16, + 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, + 0x65, 0x6f, 0x72, 0x67, 0x56, 0x32, 0x12, 0x31, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, + 0x18, 0x01, 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, 0x52, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x12, 0x34, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, - 0x74, 0x18, 0x03, 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, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x86, - 0x01, 0x0a, 0x0f, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, - 0x65, 0x72, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, - 0x02, 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, 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, 0xc7, 0x9a, 0x01, 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, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x26, 0x0a, 0x0e, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, - 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x69, - 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, - 0x02, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x6f, 0x73, 0x12, 0x20, 0x0a, - 0x0b, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x72, 0x69, 0x66, 0x74, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, 0x72, 0x69, 0x66, 0x74, 0x12, - 0x35, 0x0a, 0x08, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, - 0x65, 0x74, 0x61, 0x2e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x52, 0x08, 0x65, 0x74, - 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x12, 0x34, 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, - 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, - 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x80, 0x01, 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, 0x0a, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x35, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 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, - 0x6b, 0x0a, 0x12, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x5f, 0x68, 0x65, 0x61, 0x64, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x73, 0x48, 0x65, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 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, 0x6e, 0x0a, 0x13, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, 0x74, 0x61, 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, 0x88, 0x01, 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, 0x0d, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x32, 0x52, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x2f, 0x0a, 0x05, 0x61, 0x66, 0x74, + 0x65, 0x72, 0x18, 0x02, 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, 0x52, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x12, 0x34, 0x0a, 0x05, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x18, 0x03, 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, 0x52, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, + 0x22, 0x44, 0x0a, 0x0a, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x36, + 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, + 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x22, 0x86, 0x01, 0x0a, 0x0f, 0x42, 0x6c, 0x6f, 0x63, 0x6b, + 0x49, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x66, 0x69, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, + 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 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, + 0xff, 0x9b, 0x01, 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, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x26, 0x0a, + 0x0e, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x02, 0x6f, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x64, + 0x72, 0x69, 0x66, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x63, 0x6c, 0x6f, 0x63, + 0x6b, 0x5f, 0x64, 0x72, 0x69, 0x66, 0x74, 0x12, 0x35, 0x0a, 0x08, 0x65, 0x74, 0x68, 0x65, 0x72, + 0x65, 0x75, 0x6d, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x74, 0x68, 0x65, + 0x72, 0x65, 0x75, 0x6d, 0x52, 0x08, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x12, 0x34, + 0x0a, 0x06, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, + 0x2e, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, + 0x62, 0x65, 0x6c, 0x73, 0x12, 0x80, 0x01, 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, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 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, 0x6b, 0x0a, 0x12, 0x65, 0x74, 0x68, 0x5f, 0x76, + 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6e, - 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x44, 0x61, 0x74, 0x61, 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, 0x9a, 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, 0x0e, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, - 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x44, - 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x2d, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, + 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, 0x61, 0x64, 0x44, + 0x61, 0x74, 0x61, 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, - 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x45, 0x43, 0x4b, 0x50, - 0x4f, 0x49, 0x4e, 0x54, 0x12, 0x7f, 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, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, - 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x44, 0x61, 0x74, 0x61, 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, 0x9f, 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, 0x10, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, - 0x44, 0x61, 0x74, 0x61, 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, 0x65, 0x0a, 0x13, 0x6d, 0x65, 0x6d, 0x70, 0x6f, - 0x6f, 0x6c, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x4d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x13, 0x4d, 0x45, 0x4d, 0x50, 0x4f, - 0x4f, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x6e, - 0x0a, 0x13, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, - 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x78, 0x61, + 0x48, 0x45, 0x41, 0x44, 0x12, 0x6e, 0x0a, 0x13, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2f, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, + 0x74, 0x61, 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, 0x88, 0x01, 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, 0x0d, 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, 0x32, 0x42, 0x65, 0x61, - 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, 0x74, 0x61, 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, 0x7c, - 0x0a, 0x18, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, - 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x33, 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, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, - 0x65, 0x44, 0x61, 0x74, 0x61, 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, 0x8d, 0x01, 0x0a, - 0x1e, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, 0x6f, - 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x18, - 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 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, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, - 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, 0x44, 0x61, 0x74, 0x61, 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, 0x7a, 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, 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 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, 0x42, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x44, 0x61, - 0x74, 0x61, 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, 0x97, 0x01, 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, 0x16, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 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, 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, 0x88, 0x01, 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, 0x18, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, - 0x74, 0x61, 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, 0x73, 0x0a, - 0x15, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x68, - 0x65, 0x61, 0x64, 0x5f, 0x76, 0x32, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x78, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, + 0x44, 0x61, 0x74, 0x61, 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, + 0x9a, 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, 0x0e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 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, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, 0x61, 0x64, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x76, 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, 0x1a, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x31, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, - 0x32, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x21, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x44, 0x61, 0x74, 0x61, 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, 0x7f, 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, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x34, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, + 0x67, 0x44, 0x61, 0x74, 0x61, 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, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x56, 0x32, 0x12, 0x90, 0x01, 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, 0x1b, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, - 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0xa2, 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, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 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, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x87, 0x01, 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, 0x1d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x56, 0x32, 0x44, 0x61, 0x74, - 0x61, 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, 0xa7, 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, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, - 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, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, - 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, 0x6d, 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, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 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, 0x4d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x76, 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, - 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x84, 0x01, - 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, - 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x76, 0x32, 0x18, 0x21, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, - 0x6f, 0x69, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x95, 0x01, 0x0a, 0x21, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, - 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, - 0x65, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x5f, 0x76, 0x32, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x3a, 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, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, - 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x2c, + 0x53, 0x5f, 0x43, 0x48, 0x41, 0x49, 0x4e, 0x5f, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x12, 0x9f, 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, 0x10, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 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, 0x52, 0x45, 0x4f, 0x52, 0x47, 0x5f, 0x56, 0x32, 0x12, 0xa2, 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, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x78, + 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, + 0x65, 0x0a, 0x13, 0x6d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 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, 0x44, 0x61, 0x74, 0x61, 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, 0xa2, 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, 0x24, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x3f, 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, 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, 0x44, 0x61, - 0x74, 0x61, 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, 0x99, 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, 0x25, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 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, 0x44, 0x61, 0x74, - 0x61, 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, 0x85, 0x01, 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, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, - 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x44, 0x61, 0x74, 0x61, - 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, 0xb2, 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, 0x27, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x43, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x4c, 0x53, - 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x67, - 0x65, 0x44, 0x61, 0x74, 0x61, 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, - 0xae, 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, 0x28, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, + 0x00, 0x52, 0x13, 0x4d, 0x45, 0x4d, 0x50, 0x4f, 0x4f, 0x4c, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, + 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x6e, 0x0a, 0x13, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, + 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x12, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x2f, 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, 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, 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, - 0x12, 0x8e, 0x01, 0x0a, 0x1e, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, - 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, - 0x77, 0x61, 0x6c, 0x18, 0x29, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, - 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, - 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x29, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, + 0x44, 0x61, 0x74, 0x61, 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, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, - 0x4c, 0x12, 0x82, 0x01, 0x0a, 0x1a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, - 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, - 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, - 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, - 0x25, 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, 0x42, 0x5f, 0x53, - 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x88, 0x01, 0x0a, 0x1f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x70, 0x72, 0x69, 0x6e, 0x74, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x6c, 0x61, 0x73, - 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x3c, 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, 0x42, 0x6c, 0x6f, - 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6c, 0x61, 0x73, - 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, - 0x52, 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, 0x4c, 0x4f, - 0x43, 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, - 0x4e, 0x12, 0x82, 0x01, 0x0a, 0x1a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, - 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, - 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, - 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, - 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, - 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x6d, 0x0a, 0x16, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, - 0x5f, 0x70, 0x32, 0x70, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x2d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 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, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x50, 0x32, 0x50, 0x41, 0x74, 0x74, 0x65, - 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x16, 0x42, - 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x50, 0x32, 0x50, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, - 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x71, 0x0a, 0x14, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, - 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x64, 0x75, 0x74, 0x79, 0x18, 0x2e, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, 0x75, 0x74, - 0x79, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, - 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, - 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, 0x12, 0xb1, 0x01, 0x0a, 0x2a, 0x65, 0x74, 0x68, - 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, - 0x5f, 0x65, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x65, - 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x2f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, + 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x12, 0x7c, 0x0a, 0x18, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, + 0x5f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, + 0x63, 0x65, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 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, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, + 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 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, 0x8d, 0x01, 0x0a, 0x1e, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, + 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, + 0x65, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 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, 0x44, + 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, + 0x4f, 0x72, 0x67, 0x44, 0x61, 0x74, 0x61, 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, 0x7a, 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, + 0x15, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 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, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, + 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x44, 0x61, 0x74, 0x61, 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, 0x97, 0x01, 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, 0x16, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 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, 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, 0x88, 0x01, 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, 0x18, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x73, 0x0a, 0x15, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x76, 0x32, 0x18, 0x19, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, 0x61, 0x64, + 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x76, 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, 0x1a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x90, 0x01, 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, 0x1b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 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, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, + 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0xa2, 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, + 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, + 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x87, 0x01, 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, 0x1d, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x36, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, + 0x6f, 0x72, 0x67, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0xa7, 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, 0x1e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, + 0x6f, 0x66, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 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, 0x6d, + 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, 0x1f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, + 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, 0x4d, 0x65, 0x6d, 0x70, 0x6f, + 0x6f, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, + 0x61, 0x74, 0x61, 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, 0x76, 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, 0x20, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x32, 0x42, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x45, 0x6c, 0x61, 0x62, 0x6f, 0x72, - 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, - 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x35, 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, 0x4c, 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, - 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x69, 0x0a, 0x15, - 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x61, 0x64, 0x64, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x30, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x78, 0x61, + 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, + 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, 0x84, 0x01, 0x0a, 0x1b, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, + 0x5f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, + 0x63, 0x65, 0x5f, 0x76, 0x32, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, - 0x61, 0x63, 0x65, 0x41, 0x64, 0x64, 0x50, 0x65, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, - 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x41, - 0x44, 0x44, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x12, 0x72, 0x0a, 0x18, 0x6c, 0x69, 0x62, 0x70, 0x32, - 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x70, - 0x65, 0x65, 0x72, 0x18, 0x31, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, + 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, + 0x74, 0x61, 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, 0x95, 0x01, 0x0a, + 0x21, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x64, 0x65, 0x62, 0x75, 0x67, 0x5f, 0x66, 0x6f, + 0x72, 0x6b, 0x5f, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6f, 0x72, 0x67, 0x5f, + 0x76, 0x32, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 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, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, + 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, 0x56, 0x32, + 0x44, 0x61, 0x74, 0x61, 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, 0xa2, 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, 0x23, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 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, 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, 0x44, 0x61, 0x74, 0x61, 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, 0xa2, 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, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, - 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x65, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, - 0x00, 0x52, 0x18, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, - 0x52, 0x45, 0x4d, 0x4f, 0x56, 0x45, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x12, 0x69, 0x0a, 0x15, 0x6c, - 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x76, - 0x5f, 0x72, 0x70, 0x63, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x52, 0x65, 0x63, 0x76, 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, - 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, - 0x43, 0x56, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x69, 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, - 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x70, 0x63, 0x18, - 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 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, 0x44, 0x61, 0x74, 0x61, 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, 0x99, + 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, 0x25, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 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, 0x44, 0x61, 0x74, 0x61, 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, 0x85, 0x01, 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, 0x26, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x36, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x65, 0x70, + 0x6f, 0x73, 0x69, 0x74, 0x44, 0x61, 0x74, 0x61, 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, 0xb2, 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, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x42, 0x4c, 0x53, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x44, 0x61, 0x74, 0x61, 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, 0xae, 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, 0x28, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 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, 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, + 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, 0x12, 0x8e, 0x01, 0x0a, 0x1e, 0x65, 0x74, 0x68, + 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x18, 0x29, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x39, 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, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x57, 0x69, + 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x29, + 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, 0x57, + 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, 0x4c, 0x12, 0x82, 0x01, 0x0a, 0x1a, 0x65, 0x74, + 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, + 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, + 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, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, + 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x25, 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, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x88, + 0x01, 0x0a, 0x1f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x5f, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 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, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x42, + 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, + 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, + 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x82, 0x01, 0x0a, 0x1a, 0x65, 0x74, + 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x62, + 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, + 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, + 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, + 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x6d, + 0x0a, 0x16, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x70, 0x32, 0x70, 0x5f, 0x61, 0x74, 0x74, + 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x2d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, + 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, 0x42, 0x65, 0x61, 0x63, 0x6f, + 0x6e, 0x50, 0x32, 0x50, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, + 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x16, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x50, 0x32, + 0x50, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x71, 0x0a, + 0x14, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, + 0x5f, 0x64, 0x75, 0x74, 0x79, 0x18, 0x2e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x50, 0x72, 0x6f, + 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, 0x75, 0x74, 0x79, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, + 0x1f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, + 0x56, 0x31, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, + 0x12, 0xb1, 0x01, 0x0a, 0x2a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, + 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, + 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x2f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x53, 0x65, 0x6e, - 0x64, 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, - 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x52, 0x50, - 0x43, 0x12, 0x5e, 0x0a, 0x11, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, - 0x65, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x34, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, - 0x72, 0x61, 0x63, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x11, - 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x4a, 0x4f, 0x49, - 0x4e, 0x12, 0x6d, 0x0a, 0x16, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, - 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x35, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x33, 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, 0x4c, 0x69, - 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x16, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, - 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, 0x44, - 0x12, 0x77, 0x0a, 0x19, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, - 0x5f, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x36, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, + 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x32, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x45, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x74, 0x65, + 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x35, 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, 0x4c, + 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x69, 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, + 0x72, 0x61, 0x63, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x30, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x31, 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, - 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x44, 0x69, 0x73, 0x63, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1a, 0x4c, - 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x43, - 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, 0x44, 0x12, 0x7e, 0x0a, 0x1c, 0x6c, 0x69, 0x62, - 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, - 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x38, 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, 0x4c, 0x69, 0x62, 0x50, - 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x4d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1c, 0x4c, 0x49, 0x42, - 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, - 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x12, 0x78, 0x0a, 0x1a, 0x6c, 0x69, 0x62, - 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, - 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1a, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, - 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x55, 0x53, 0x12, 0x92, 0x01, 0x0a, 0x23, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, - 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x39, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x3e, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, + 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x41, 0x64, 0x64, 0x50, 0x65, + 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, + 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x12, + 0x72, 0x0a, 0x18, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, + 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x31, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x34, 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, 0x4c, 0x69, - 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, - 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, 0x74, - 0x61, 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, - 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x12, 0xa4, 0x01, 0x0a, 0x29, 0x6c, 0x69, 0x62, - 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, - 0x73, 0x75, 0x62, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, - 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, - 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, - 0x74, 0x61, 0x48, 0x00, 0x52, 0x29, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, - 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, - 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, - 0x92, 0x01, 0x0a, 0x23, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, - 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, - 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x3b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, + 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, + 0x65, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x18, 0x4c, 0x49, 0x42, 0x50, 0x32, + 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4d, 0x4f, 0x56, 0x45, 0x5f, 0x50, + 0x45, 0x45, 0x52, 0x12, 0x69, 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, + 0x61, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x76, 0x5f, 0x72, 0x70, 0x63, 0x18, 0x32, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x31, 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, 0x4c, + 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x63, 0x76, 0x52, 0x50, + 0x43, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, + 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, 0x43, 0x56, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x69, + 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x73, + 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x70, 0x63, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x6c, - 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, - 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, - 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, - 0x45, 0x43, 0x41, 0x52, 0x1a, 0x80, 0x03, 0x0a, 0x08, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, - 0x6d, 0x12, 0x3b, 0x0a, 0x07, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, - 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x4e, 0x65, - 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x52, 0x07, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, 0x41, - 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, - 0x65, 0x74, 0x61, 0x2e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x78, 0x65, - 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x41, 0x0a, 0x09, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, - 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x73, 0x65, - 0x6e, 0x73, 0x75, 0x73, 0x1a, 0x2d, 0x0a, 0x07, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, - 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, - 0x61, 0x6d, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x02, 0x69, 0x64, 0x1a, 0x33, 0x0a, 0x09, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x26, 0x0a, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x49, 0x44, 0x52, - 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x1a, 0x4d, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x73, - 0x65, 0x6e, 0x73, 0x75, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, - 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x69, - 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, - 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, - 0x73, 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, 0x1a, 0x49, 0x0a, 0x24, 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, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, - 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x1a, 0x4d, 0x0a, - 0x26, 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, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, - 0x18, 0x01, 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, 0x1a, 0x49, 0x0a, 0x24, + 0x54, 0x72, 0x61, 0x63, 0x65, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, + 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, + 0x5f, 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x5e, 0x0a, 0x11, 0x6c, 0x69, 0x62, + 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x34, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x4a, 0x6f, 0x69, 0x6e, + 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x11, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, + 0x52, 0x41, 0x43, 0x45, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x12, 0x6d, 0x0a, 0x16, 0x6c, 0x69, 0x62, + 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, + 0x74, 0x65, 0x64, 0x18, 0x35, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, + 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, + 0x52, 0x16, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x43, + 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, 0x44, 0x12, 0x77, 0x0a, 0x19, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x36, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, + 0x61, 0x63, 0x65, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x44, + 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1a, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, + 0x41, 0x43, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x43, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, + 0x44, 0x12, 0x7e, 0x0a, 0x1c, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, + 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, + 0x61, 0x18, 0x37, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, + 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x61, 0x74, + 0x61, 0x48, 0x00, 0x52, 0x1c, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, + 0x45, 0x5f, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, + 0x41, 0x12, 0x78, 0x0a, 0x1a, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, + 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x38, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, + 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, + 0x1a, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x48, 0x41, + 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x12, 0x92, 0x01, 0x0a, 0x23, + 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, + 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 0x18, 0x39, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, + 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, 0x42, + 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, + 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, + 0x12, 0xa4, 0x01, 0x0a, 0x29, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, + 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x65, 0x61, 0x63, + 0x6f, 0x6e, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x3a, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, + 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, + 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x29, 0x4c, 0x49, + 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, + 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x45, + 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x92, 0x01, 0x0a, 0x23, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, + 0x75, 0x62, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, + 0x3b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, + 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, + 0x72, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, + 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, + 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x70, 0x0a, 0x11, + 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, + 0x73, 0x18, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 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, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x6f, 0x72, 0x73, 0x44, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x23, 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, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, 0x52, 0x53, 0x1a, 0x80, + 0x03, 0x0a, 0x08, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x12, 0x3b, 0x0a, 0x07, 0x6e, + 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x45, + 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x52, + 0x07, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, 0x41, 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x74, + 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x09, 0x63, + 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, + 0x2e, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, + 0x73, 0x75, 0x73, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x1a, 0x2d, + 0x0a, 0x07, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x0e, 0x0a, + 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x02, 0x69, 0x64, 0x1a, 0x33, 0x0a, + 0x09, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x26, 0x0a, 0x07, 0x66, 0x6f, + 0x72, 0x6b, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x49, 0x44, 0x52, 0x07, 0x66, 0x6f, 0x72, 0x6b, 0x5f, + 0x69, 0x64, 0x1a, 0x4d, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x12, + 0x26, 0x0a, 0x0e, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x69, 0x6d, 0x70, 0x6c, 0x65, 0x6d, 0x65, + 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 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, 0x1a, 0x49, 0x0a, 0x24, 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, + 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x1a, 0x4d, 0x0a, 0x26, 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, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x1a, 0x88, 0x03, 0x0a, 0x24, 0x41, 0x64, 0x64, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, - 0x4d, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x35, 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, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x4d, - 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, - 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, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x1e, 0x0a, - 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x21, 0x0a, - 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, - 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, - 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, - 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x13, 0x61, 0x74, - 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, - 0x72, 0x1a, 0x96, 0x03, 0x0a, 0x26, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x41, 0x74, 0x74, 0x65, 0x73, - 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 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, 0x20, - 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x1a, 0x49, 0x0a, 0x24, 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, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, + 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, + 0x68, 0x1a, 0x4d, 0x0a, 0x26, 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, 0x12, 0x23, 0x0a, 0x05, 0x65, + 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, + 0x1a, 0x88, 0x03, 0x0a, 0x24, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, + 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x4d, 0x0a, 0x06, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x44, 0x61, 0x74, 0x61, + 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x4d, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x44, 0x61, 0x74, 0x61, 0x52, + 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, + 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, + 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, + 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x4a, 0x0a, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x1a, 0x96, 0x03, 0x0a, 0x26, + 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x56, 0x32, 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, 0x20, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, + 0x18, 0x03, 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, 0x23, 0x0a, 0x05, 0x65, 0x70, + 0x6f, 0x63, 0x68, 0x18, 0x04, 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, + 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, + 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, + 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, + 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x56, 0x32, 0x52, + 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, + 0x61, 0x74, 0x6f, 0x72, 0x1a, 0x97, 0x01, 0x0a, 0x1d, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, + 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, + 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, + 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, + 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x9f, + 0x01, 0x0a, 0x1f, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, + 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, 0x61, 0x64, 0x56, 0x32, 0x44, 0x61, + 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, + 0x02, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, + 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x1a, 0x98, 0x01, 0x0a, 0x1e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, + 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, + 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, + 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, + 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, + 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xa0, 0x01, 0x0a, 0x20, + 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, + 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, - 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x13, - 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x6f, 0x72, 0x56, 0x32, 0x52, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, - 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x1a, 0x97, 0x01, 0x0a, 0x1d, 0x41, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x20, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, + 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4b, + 0x0a, 0x26, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, + 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, + 0x45, 0x78, 0x69, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, + 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, + 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x1a, 0x4f, 0x0a, 0x28, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, 0x61, 0x64, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, + 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, + 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, + 0x18, 0x01, 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, 0x1a, 0x51, 0x0a, 0x2c, + 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x1a, + 0x55, 0x0a, 0x2e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, + 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, + 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, + 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x1a, 0x9d, 0x01, 0x0a, 0x23, 0x41, 0x64, 0x64, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, + 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, + 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, + 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, + 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xa5, 0x01, 0x0a, 0x25, 0x41, 0x64, 0x64, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, + 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, + 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, + 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xb3, + 0x01, 0x0a, 0x39, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, + 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, @@ -8362,703 +8585,625 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x9f, 0x01, 0x0a, 0x1f, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x65, - 0x61, 0x64, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, - 0x68, 0x18, 0x01, 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, 0x02, 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, - 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, - 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, - 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x98, 0x01, 0x0a, 0x1e, 0x41, 0x64, 0x64, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, - 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, - 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, - 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, - 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x33, 0x0a, 0x0b, - 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x1a, 0xa0, 0x01, 0x0a, 0x20, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x63, 0x6b, - 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, - 0x01, 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, 0x02, 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, 0x35, 0x0a, - 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x4b, 0x0a, 0x26, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, - 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, - 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, - 0x68, 0x1a, 0x4f, 0x0a, 0x28, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, - 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, - 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, - 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x1a, 0x51, 0x0a, 0x2c, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, - 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x44, 0x61, - 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, - 0x65, 0x70, 0x6f, 0x63, 0x68, 0x1a, 0x55, 0x0a, 0x2e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x46, 0x69, - 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, - 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, - 0x18, 0x01, 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, 0x1a, 0x9d, 0x01, 0x0a, - 0x23, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, - 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, - 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, - 0x74, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, - 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xa5, 0x01, 0x0a, - 0x25, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x6f, 0x72, 0x67, - 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, - 0x01, 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, 0x02, 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, 0x35, 0x0a, - 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xb3, 0x01, 0x0a, 0x39, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, + 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xbb, 0x01, 0x0a, 0x3b, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, - 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, - 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, - 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, - 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x70, - 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xbb, 0x01, 0x0a, 0x3b, 0x41, - 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, - 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, - 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, - 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x9f, 0x01, 0x0a, 0x2d, 0x41, 0x64, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, - 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, - 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x44, 0x61, 0x74, 0x61, 0x12, 0x6e, 0x0a, 0x0c, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x4a, 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, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xa3, 0x01, 0x0a, 0x2f, 0x41, - 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x70, - 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4c, 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, + 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, + 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x35, 0x0a, 0x0b, 0x70, + 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x1a, 0x9f, 0x01, 0x0a, 0x2d, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, - 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, - 0x74, 0x61, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x1a, 0xad, 0x02, 0x0a, 0x12, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, - 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x12, 0x31, 0x0a, 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, - 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x0d, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2e, 0x0a, 0x0c, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x0c, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x48, 0x0a, 0x1f, 0x72, 0x65, + 0x44, 0x61, 0x74, 0x61, 0x12, 0x6e, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 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, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, + 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xa3, 0x01, 0x0a, 0x2f, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, + 0x6f, 0x66, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x70, 0x0a, 0x0c, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4c, + 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, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x41, 0x6e, 0x64, 0x50, 0x72, 0x6f, 0x6f, 0x66, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x69, + 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0c, 0x63, 0x6f, + 0x6e, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0xad, 0x02, 0x0a, 0x12, 0x46, + 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, + 0x74, 0x12, 0x31, 0x0a, 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x70, 0x6f, + 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, + 0x70, 0x6f, 0x63, 0x68, 0x12, 0x2e, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, + 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, + 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x48, 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, 0x03, 0x20, 0x01, 0x28, 0x04, 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, 0x30, + 0x0a, 0x13, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 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, 0x05, 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, 0xef, 0x02, 0x0a, 0x14, 0x46, + 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, + 0x74, 0x56, 0x32, 0x12, 0x33, 0x0a, 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, + 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x78, 0x61, 0x74, + 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x56, 0x32, 0x52, 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x30, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x0c, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x6c, 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, 0x03, 0x20, - 0x01, 0x28, 0x04, 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, 0x30, 0x0a, 0x13, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, - 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x04, 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, 0x05, 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, 0xef, 0x02, 0x0a, 0x14, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, - 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x56, 0x32, 0x12, 0x33, 0x0a, 0x0d, 0x72, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x0d, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x56, 0x32, 0x52, - 0x0d, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x30, - 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, - 0x56, 0x32, 0x52, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x73, 0x6c, 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, 0x03, 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, - 0x04, 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, 0x05, 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, 0x65, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, - 0x6f, 0x69, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3f, 0x0a, 0x08, 0x53, 0x6e, 0x61, 0x70, - 0x73, 0x68, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, + 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, 0x04, 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, + 0x05, 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, 0x65, 0x0a, 0x22, + 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, + 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x44, 0x61, + 0x74, 0x61, 0x12, 0x3f, 0x0a, 0x08, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, + 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x1a, 0x69, 0x0a, 0x24, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, + 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, + 0x68, 0x6f, 0x69, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x41, 0x0a, 0x08, 0x53, + 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, + 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, + 0x6f, 0x74, 0x56, 0x32, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x1a, 0xa1, + 0x01, 0x0a, 0x27, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, + 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, + 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x06, 0x62, 0x65, + 0x66, 0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, - 0x08, 0x73, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x1a, 0x69, 0x0a, 0x24, 0x41, 0x64, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, - 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x56, 0x32, 0x44, 0x61, 0x74, - 0x61, 0x12, 0x41, 0x0a, 0x08, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, - 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, - 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x08, 0x73, 0x6e, 0x61, 0x70, - 0x73, 0x68, 0x6f, 0x74, 0x1a, 0xa1, 0x01, 0x0a, 0x27, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, - 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, 0x44, 0x61, 0x74, 0x61, - 0x12, 0x3b, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, - 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, - 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x39, 0x0a, - 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x46, - 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, - 0x74, 0x52, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x1a, 0xa7, 0x01, 0x0a, 0x29, 0x41, 0x64, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, - 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, - 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3d, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, + 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x39, 0x0a, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, - 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x06, 0x62, - 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, 0x3b, 0x0a, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, - 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, - 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x05, 0x61, 0x66, 0x74, - 0x65, 0x72, 0x1a, 0x6b, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, - 0x74, 0x74, 0x65, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, - 0x68, 0x18, 0x01, 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, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x1a, - 0xf2, 0x01, 0x0a, 0x20, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4d, 0x65, - 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x44, 0x61, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x6f, 0x6d, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x66, 0x72, 0x6f, 0x6d, 0x12, 0x0e, 0x0a, 0x02, - 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x74, 0x6f, 0x12, 0x14, 0x0a, 0x05, - 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x6e, 0x6f, 0x6e, - 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, - 0x12, 0x10, 0x0a, 0x03, 0x67, 0x61, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x67, - 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, - 0x18, 0x08, 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, 0x09, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, - 0x73, 0x69, 0x7a, 0x65, 0x1a, 0x9a, 0x05, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x4d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x68, - 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, - 0x12, 0x0a, 0x04, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x66, - 0x72, 0x6f, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x02, 0x74, 0x6f, 0x12, 0x32, 0x0a, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x04, 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, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x70, - 0x72, 0x69, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x61, 0x73, 0x5f, - 0x70, 0x72, 0x69, 0x63, 0x65, 0x12, 0x2e, 0x0a, 0x03, 0x67, 0x61, 0x73, 0x18, 0x06, 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, 0x03, 0x67, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, - 0x69, 0x7a, 0x65, 0x18, 0x08, 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, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x30, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, - 0x0a, 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, 0x33, 0x32, 0x56, 0x61, - 0x6c, 0x75, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x67, 0x61, 0x73, - 0x5f, 0x74, 0x69, 0x70, 0x5f, 0x63, 0x61, 0x70, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x67, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x70, 0x5f, 0x63, 0x61, 0x70, 0x12, 0x20, 0x0a, 0x0b, 0x67, - 0x61, 0x73, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x67, 0x61, 0x73, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x12, 0x38, 0x0a, - 0x08, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x0d, 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, 0x08, 0x62, - 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x12, 0x2a, 0x0a, 0x10, 0x62, 0x6c, 0x6f, 0x62, 0x5f, - 0x67, 0x61, 0x73, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x18, 0x0e, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x10, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x66, 0x65, 0x65, 0x5f, - 0x63, 0x61, 0x70, 0x12, 0x20, 0x0a, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x68, 0x61, 0x73, 0x68, - 0x65, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x68, - 0x61, 0x73, 0x68, 0x65, 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, - 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, - 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x3a, 0x0a, 0x18, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, - 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, - 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x1a, 0x89, 0x02, 0x0a, 0x1e, 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, - 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, - 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, - 0x74, 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, 0x1e, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, - 0x74, 0x12, 0x2e, 0x0a, 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x74, - 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x12, 0x3a, 0x0a, 0x18, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x06, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x18, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x1a, 0x8d, 0x05, - 0x0a, 0x20, 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, 0x56, 0x32, 0x44, 0x61, + 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x52, 0x05, 0x61, 0x66, 0x74, + 0x65, 0x72, 0x1a, 0xa7, 0x01, 0x0a, 0x29, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, + 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x44, 0x65, 0x62, 0x75, 0x67, 0x46, 0x6f, 0x72, 0x6b, 0x43, + 0x68, 0x6f, 0x69, 0x63, 0x65, 0x52, 0x65, 0x4f, 0x72, 0x67, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, + 0x12, 0x3d, 0x0a, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x25, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, + 0x74, 0x61, 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, + 0x70, 0x73, 0x68, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x06, 0x62, 0x65, 0x66, 0x6f, 0x72, 0x65, 0x12, + 0x3b, 0x0a, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, + 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, + 0x2e, 0x46, 0x6f, 0x72, 0x6b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x6e, 0x61, 0x70, 0x73, + 0x68, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x05, 0x61, 0x66, 0x74, 0x65, 0x72, 0x1a, 0x6b, 0x0a, 0x22, + 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, + 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 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, 0x1e, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, - 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, - 0x6f, 0x6f, 0x74, 0x12, 0x4c, 0x0a, 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x1a, 0xf2, 0x01, 0x0a, 0x20, 0x41, 0x64, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4d, 0x65, 0x6d, 0x70, 0x6f, 0x6f, 0x6c, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x12, + 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, + 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x6f, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x66, 0x72, 0x6f, 0x6d, 0x12, 0x0e, 0x0a, 0x02, 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x02, 0x74, 0x6f, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x67, 0x61, + 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x67, 0x61, 0x73, 0x12, 0x14, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 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, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, + 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x1a, 0x9a, + 0x05, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4d, 0x65, 0x6d, + 0x70, 0x6f, 0x6f, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, + 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x66, 0x72, 0x6f, + 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x66, 0x72, 0x6f, 0x6d, 0x12, 0x0e, 0x0a, + 0x02, 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x74, 0x6f, 0x12, 0x32, 0x0a, + 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x04, 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, 0x05, 0x6e, 0x6f, 0x6e, 0x63, + 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x70, 0x72, 0x69, 0x63, 0x65, 0x12, + 0x2e, 0x0a, 0x03, 0x67, 0x61, 0x73, 0x18, 0x06, 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, 0x03, 0x67, 0x61, 0x73, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 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, 0x09, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0e, 0x63, 0x61, 0x6c, 0x6c, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x12, 0x30, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x0a, 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, 0x12, 0x74, + 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x67, 0x61, 0x73, 0x5f, 0x74, 0x69, 0x70, 0x5f, 0x63, + 0x61, 0x70, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x67, 0x61, 0x73, 0x5f, 0x74, 0x69, + 0x70, 0x5f, 0x63, 0x61, 0x70, 0x12, 0x20, 0x0a, 0x0b, 0x67, 0x61, 0x73, 0x5f, 0x66, 0x65, 0x65, + 0x5f, 0x63, 0x61, 0x70, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x67, 0x61, 0x73, 0x5f, + 0x66, 0x65, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x12, 0x38, 0x0a, 0x08, 0x62, 0x6c, 0x6f, 0x62, 0x5f, + 0x67, 0x61, 0x73, 0x18, 0x0d, 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, 0x08, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, + 0x73, 0x12, 0x2a, 0x0a, 0x10, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x66, 0x65, + 0x65, 0x5f, 0x63, 0x61, 0x70, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x62, 0x6c, 0x6f, + 0x62, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x63, 0x61, 0x70, 0x12, 0x20, 0x0a, + 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x65, 0x73, 0x18, 0x0f, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x65, 0x73, 0x12, + 0x2e, 0x0a, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, + 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x62, 0x6c, 0x6f, + 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, + 0x3a, 0x0a, 0x18, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, + 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x11, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x18, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, + 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x1a, 0x89, 0x02, 0x0a, 0x1e, + 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, 0x44, 0x61, 0x74, 0x61, 0x12, 0x21, + 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x52, 0x05, 0x65, 0x70, 0x6f, 0x63, + 0x68, 0x12, 0x1e, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x0a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 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, 0x1e, 0x0a, 0x0a, 0x62, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x12, 0x2e, 0x0a, 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x12, 0x58, 0x0a, 0x18, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x06, 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, 0x18, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, - 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x12, 0x6e, 0x0a, 0x23, 0x74, + 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x3a, 0x0a, 0x18, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, - 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, - 0x65, 0x64, 0x18, 0x07, 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, 0x23, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, - 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x3e, 0x0a, 0x0b, 0x74, - 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x08, 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, 0x0b, - 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x12, 0x54, 0x0a, 0x16, 0x74, + 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x18, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x1a, 0x8d, 0x05, 0x0a, 0x20, 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, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 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, 0x1e, 0x0a, + 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x12, 0x4c, 0x0a, + 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x63, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x05, 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, 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x58, 0x0a, 0x18, 0x74, + 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, + 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x06, 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, 0x18, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, + 0x62, 0x79, 0x74, 0x65, 0x73, 0x12, 0x6e, 0x0a, 0x23, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, + 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x07, 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, 0x23, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, - 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, 0x09, 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, 0x16, 0x74, 0x6f, 0x74, 0x61, 0x6c, - 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, - 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x77, - 0x68, 0x65, 0x6e, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x18, 0x0a, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x18, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x77, - 0x68, 0x65, 0x6e, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x1a, 0x5d, 0x0a, - 0x2e, 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, 0x41, 0x74, 0x74, 0x65, 0x73, - 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 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, 0x5d, 0x0a, 0x2e, - 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, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, - 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, 0x68, 0x69, 0x6e, 0x67, 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, 0x5a, 0x0a, 0x2b, 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, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, - 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 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, 0x54, 0x0a, 0x25, 0x41, 0x64, 0x64, 0x69, 0x74, + 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x3e, 0x0a, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, + 0x79, 0x74, 0x65, 0x73, 0x18, 0x08, 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, 0x0b, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, + 0x62, 0x79, 0x74, 0x65, 0x73, 0x12, 0x54, 0x0a, 0x16, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, + 0x79, 0x74, 0x65, 0x73, 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x18, + 0x09, 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, 0x16, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, + 0x5f, 0x63, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x65, 0x64, 0x12, 0x3a, 0x0a, 0x18, 0x66, + 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x77, 0x68, 0x65, 0x6e, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x66, + 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x77, 0x68, 0x65, 0x6e, 0x5f, 0x72, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x1a, 0x5d, 0x0a, 0x2e, 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, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 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, 0x61, 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, 0x42, 0x4c, 0x53, 0x54, 0x6f, - 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 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, 0xd5, 0x02, 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, 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, 0x12, 0x4a, 0x0a, 0x11, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x69, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 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, - 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, 0x12, 0x2e, 0x0a, 0x12, - 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x73, 0x69, - 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, - 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x3a, 0x0a, 0x18, - 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x65, 0x6d, - 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, - 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x65, 0x6d, - 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x1a, 0x57, 0x0a, 0x28, 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, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, + 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x65, 0x72, 0x53, 0x6c, 0x61, + 0x73, 0x68, 0x69, 0x6e, 0x67, 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, 0x5d, 0x0a, 0x2e, 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, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x53, 0x6c, 0x61, 0x73, + 0x68, 0x69, 0x6e, 0x67, 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, 0x5a, 0x0a, 0x2b, 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, 0x56, 0x6f, 0x6c, 0x75, 0x6e, 0x74, 0x61, 0x72, 0x79, 0x45, 0x78, 0x69, 0x74, 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, 0x74, 0x0a, 0x2b, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x42, - 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6c, - 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, - 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, - 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 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, - 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, 0x1a, 0xa4, 0x01, 0x0a, 0x24, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, - 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, - 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, - 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x99, 0x02, 0x0a, 0x24, - 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, - 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, - 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x3a, 0x0a, 0x09, 0x64, - 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 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, 0x09, 0x64, 0x61, - 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x76, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x65, 0x64, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x12, - 0x46, 0x0a, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, - 0x7a, 0x65, 0x18, 0x05, 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, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x65, 0x6d, 0x70, - 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x1a, 0xa9, 0x04, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x50, 0x32, 0x50, 0x41, - 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, + 0x6b, 0x1a, 0x54, 0x0a, 0x25, 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, 0x44, + 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 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, 0x61, 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, 0x42, 0x4c, 0x53, 0x54, 0x6f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, + 0x69, 0x6f, 0x6e, 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, 0xd5, 0x02, 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, 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, 0x12, 0x4a, + 0x0a, 0x11, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x5f, 0x62, 0x6c, + 0x6f, 0x63, 0x6b, 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, 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, 0x12, 0x2e, 0x0a, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, + 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x12, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, + 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x3a, 0x0a, 0x18, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, + 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x18, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, + 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x73, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x1a, 0x57, 0x0a, 0x28, 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, + 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 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, 0x74, 0x0a, 0x2b, 0x41, + 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, + 0x69, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, + 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, + 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, + 0x74, 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, 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, 0x54, 0x61, 0x72, 0x67, - 0x65, 0x74, 0x56, 0x32, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x12, 0x20, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x04, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, - 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, - 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4c, - 0x0a, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, - 0x64, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, - 0x64, 0x61, 0x74, 0x6f, 0x72, 0x56, 0x32, 0x52, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, - 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x25, 0x0a, 0x04, - 0x70, 0x65, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x04, 0x70, - 0x65, 0x65, 0x72, 0x12, 0x34, 0x0a, 0x06, 0x73, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x18, 0x08, 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, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x52, 0x06, 0x73, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x12, 0x38, 0x0a, 0x09, 0x76, 0x61, 0x6c, - 0x69, 0x64, 0x61, 0x74, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, - 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x42, - 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x65, 0x64, 0x1a, 0x84, 0x01, 0x0a, 0x1f, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, - 0x75, 0x74, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, + 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, 0x1a, 0xa4, + 0x01, 0x0a, 0x24, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, + 0x56, 0x31, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, + 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x1a, - 0x0a, 0x08, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x08, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x1a, 0x97, 0x03, 0x0a, 0x33, 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, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, - 0x74, 0x65, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x12, - 0x4a, 0x0a, 0x11, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x5f, 0x62, - 0x6c, 0x6f, 0x63, 0x6b, 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, 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, 0x4f, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x05, 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, 0x06, 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, 0x1a, 0x5f, 0x0a, 0x20, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x41, 0x64, 0x64, - 0x50, 0x65, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, + 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x35, + 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, + 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x99, 0x02, 0x0a, 0x24, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, + 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, + 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x3a, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, + 0x7a, 0x65, 0x18, 0x03, 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, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x12, 0x26, 0x0a, 0x0e, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x65, 0x64, 0x5f, 0x68, + 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x65, 0x64, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x12, 0x46, 0x0a, 0x0f, 0x64, 0x61, 0x74, + 0x61, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x05, 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, 0x0f, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x5f, 0x73, 0x69, 0x7a, + 0x65, 0x1a, 0xa9, 0x04, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, + 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x50, 0x32, 0x50, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 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, 0x20, 0x0a, 0x04, 0x73, 0x6c, + 0x6f, 0x74, 0x18, 0x03, 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, 0x23, 0x0a, 0x05, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x04, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, + 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, + 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x13, 0x61, 0x74, 0x74, 0x65, + 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x41, 0x74, 0x74, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x56, + 0x32, 0x52, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x25, 0x0a, 0x04, 0x70, 0x65, 0x65, 0x72, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x52, 0x04, 0x70, 0x65, 0x65, 0x72, 0x12, 0x34, 0x0a, + 0x06, 0x73, 0x75, 0x62, 0x6e, 0x65, 0x74, 0x18, 0x08, 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, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x73, 0x75, 0x62, + 0x6e, 0x65, 0x74, 0x12, 0x38, 0x0a, 0x09, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x64, + 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x64, 0x1a, 0x84, 0x01, + 0x0a, 0x1f, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, + 0x31, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, 0x75, 0x74, 0x79, 0x44, 0x61, 0x74, + 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, + 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, 0x1a, 0x0a, 0x08, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x73, 0x74, 0x61, 0x74, + 0x65, 0x5f, 0x69, 0x64, 0x1a, 0x97, 0x03, 0x0a, 0x33, 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, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x74, + 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 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, 0x12, 0x4a, 0x0a, 0x11, 0x70, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 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, 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, 0x4f, 0x0a, 0x06, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x05, 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, 0x06, 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, 0x1a, 0x5f, + 0x0a, 0x20, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, + 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x41, 0x64, 0x64, 0x50, 0x65, 0x65, 0x72, 0x44, 0x61, + 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, + 0x62, 0x0a, 0x23, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, + 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x65, + 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x1a, 0x5f, 0x0a, 0x20, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, + 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, 0x63, 0x76, + 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x1a, 0x5f, 0x0a, 0x20, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x53, 0x65, 0x6e, + 0x64, 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, - 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x62, 0x0a, 0x23, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x52, 0x65, - 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x65, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, - 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, - 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, - 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x5f, 0x0a, 0x20, 0x41, 0x64, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x52, 0x65, 0x63, 0x76, 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, - 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, - 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x5f, 0x0a, 0x20, 0x41, 0x64, - 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, - 0x61, 0x63, 0x65, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x50, 0x43, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, - 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, - 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x5c, 0x0a, 0x1d, 0x41, - 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, - 0x72, 0x61, 0x63, 0x65, 0x4a, 0x6f, 0x69, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, - 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, - 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, - 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x61, 0x0a, 0x22, 0x41, 0x64, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, - 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x12, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x5c, 0x0a, 0x1d, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x4a, 0x6f, + 0x69, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, + 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x1a, 0x61, 0x0a, 0x22, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, + 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x6e, + 0x65, 0x63, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x64, 0x0a, 0x25, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x44, + 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x64, 0x0a, 0x25, + 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x66, 0x0a, 0x27, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, - 0x64, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, - 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, - 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x1a, 0x66, 0x0a, 0x27, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, - 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, + 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x1a, 0x64, 0x0a, 0x25, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, + 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x64, 0x0a, 0x25, 0x41, 0x64, - 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, - 0x61, 0x63, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x44, - 0x61, 0x74, 0x61, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, - 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, - 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0x1a, 0x8d, 0x04, 0x0a, 0x2d, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, - 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, - 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, - 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, - 0x02, 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, 0x37, 0x0a, 0x0f, 0x77, 0x61, 0x6c, - 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 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, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, 0x6f, - 0x63, 0x68, 0x12, 0x34, 0x0a, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, - 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, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, - 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, - 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, - 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x40, 0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x18, 0x08, 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, 0x33, 0x32, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, - 0x7a, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, - 0x1a, 0x60, 0x0a, 0x39, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, - 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, - 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, - 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x1a, 0x60, 0x0a, 0x39, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, - 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, - 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, - 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x1a, 0xa9, 0x06, 0x0a, 0x33, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, - 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, - 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x62, 0x0a, 0x06, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x78, - 0x61, 0x74, 0x75, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x2e, 0x41, + 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x8d, 0x04, 0x0a, 0x2d, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, - 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x12, 0x62, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x4a, 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, 0x4c, 0x69, 0x62, + 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x37, 0x0a, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 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, 0x0f, 0x77, 0x61, 0x6c, + 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x34, 0x0a, 0x0e, + 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 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, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, + 0x6f, 0x74, 0x12, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, + 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, + 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x40, 0x0a, 0x0c, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 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, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x3c, 0x0a, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x1a, 0x60, 0x0a, 0x39, 0x41, 0x64, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, + 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, + 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, + 0x18, 0x01, 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, 0x1a, 0x60, 0x0a, 0x39, + 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, + 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, + 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, + 0x63, 0x68, 0x18, 0x01, 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, 0x1a, 0xa9, + 0x06, 0x0a, 0x33, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x74, 0x61, - 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, - 0x04, 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, 0x35, 0x0a, 0x0b, 0x70, - 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, - 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, - 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x56, 0x32, 0x52, 0x13, 0x61, 0x74, 0x74, - 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, - 0x12, 0x37, 0x0a, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, - 0x6f, 0x63, 0x68, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, 0x56, 0x32, 0x52, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, - 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x34, 0x0a, 0x0e, 0x77, 0x61, 0x6c, - 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, - 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x12, - 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, - 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x40, 0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x18, 0x0b, 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, 0x33, 0x32, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, - 0x7a, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, - 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, - 0x1a, 0x8d, 0x04, 0x0a, 0x2d, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, - 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, - 0x53, 0x75, 0x62, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, - 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, - 0x02, 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, 0x37, 0x0a, 0x0f, 0x77, 0x61, 0x6c, - 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 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, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, 0x6f, - 0x63, 0x68, 0x12, 0x34, 0x0a, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, - 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, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, - 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, - 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, - 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, - 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, - 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x40, 0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, - 0x18, 0x08, 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, 0x33, 0x32, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, - 0x7a, 0x65, 0x12, 0x3c, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, + 0x6f, 0x6e, 0x44, 0x61, 0x74, 0x61, 0x12, 0x62, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, + 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x41, 0x74, 0x74, + 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x61, + 0x74, 0x61, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x62, 0x0a, 0x06, 0x74, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4a, 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, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, 0x72, 0x61, + 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x65, 0x61, 0x63, 0x6f, + 0x6e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, + 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x52, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, + 0x0a, 0x04, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x03, 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, 0x23, 0x0a, 0x05, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x04, 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, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, + 0x75, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, + 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4c, 0x0a, 0x13, + 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x6f, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x6f, 0x72, 0x56, 0x32, 0x52, 0x13, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x67, + 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x12, 0x37, 0x0a, 0x0f, 0x77, 0x61, + 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x45, 0x70, 0x6f, 0x63, 0x68, + 0x56, 0x32, 0x52, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, + 0x6f, 0x63, 0x68, 0x12, 0x34, 0x0a, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0c, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x53, 0x6c, 0x6f, 0x74, 0x56, 0x32, 0x52, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, + 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, + 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x40, 0x0a, 0x0c, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x0b, 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, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x3c, 0x0a, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x1a, 0x8d, 0x04, 0x0a, 0x2d, 0x41, + 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x62, 0x50, 0x32, 0x50, 0x54, + 0x72, 0x61, 0x63, 0x65, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x75, 0x62, 0x42, 0x6c, 0x6f, + 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, + 0x65, 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x02, 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, 0x37, 0x0a, 0x0f, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, + 0x5f, 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, 0x0f, 0x77, 0x61, 0x6c, + 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x70, 0x6f, 0x63, 0x68, 0x12, 0x34, 0x0a, 0x0e, + 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 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, 0x0e, 0x77, 0x61, 0x6c, 0x6c, 0x63, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x73, 0x6c, + 0x6f, 0x74, 0x12, 0x35, 0x0a, 0x0b, 0x70, 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x50, + 0x72, 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x52, 0x0b, 0x70, 0x72, + 0x6f, 0x70, 0x61, 0x67, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x08, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x54, 0x72, 0x61, 0x63, 0x65, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, + 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, + 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, + 0x6c, 0x75, 0x65, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x40, 0x0a, 0x0c, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x08, 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, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0c, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x3c, 0x0a, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x1a, 0x44, 0x0a, 0x1d, 0x41, 0x64, + 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x45, 0x74, 0x68, 0x56, 0x31, 0x56, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x44, 0x61, 0x74, 0x61, 0x12, 0x23, 0x0a, 0x05, 0x65, + 0x70, 0x6f, 0x63, 0x68, 0x18, 0x01, 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, 0x42, 0x10, 0x0a, 0x0e, 0x41, 0x64, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x44, 0x61, 0x74, 0x61, 0x22, 0xca, 0x09, 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, @@ -9142,7 +9287,7 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 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, 0xcb, 0x11, 0x0a, 0x05, + 0x65, 0x74, 0x61, 0x52, 0x06, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x22, 0xf4, 0x11, 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, @@ -9150,7 +9295,7 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 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, 0xd1, 0x10, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x24, + 0x09, 0x52, 0x02, 0x69, 0x64, 0x22, 0xfa, 0x10, 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, @@ -9283,349 +9428,357 @@ var file_pkg_proto_xatu_event_ingester_proto_rawDesc = []byte{ 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x32, 0x12, 0x27, 0x0a, 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x4c, 0x4f, 0x42, 0x5f, - 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x10, 0x33, 0x22, 0xca, 0x29, 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, + 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x10, 0x33, 0x12, 0x27, 0x0a, 0x23, 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, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, 0x52, 0x53, + 0x10, 0x34, 0x22, 0x9e, 0x2a, 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, 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, + 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, 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, 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, + 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, 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, 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, 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, - 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, + 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, 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, 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, + 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, 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, 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, + 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, 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, + 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, 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, 0x12, 0x6e, 0x0a, 0x1e, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x5f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x18, 0x21, + 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, 0x12, 0x6e, 0x0a, 0x1e, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, + 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x77, 0x69, 0x74, + 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x74, 0x68, + 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x56, 0x32, 0x48, 0x00, 0x52, 0x29, 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, 0x57, 0x49, 0x54, 0x48, 0x44, + 0x52, 0x41, 0x57, 0x41, 0x4c, 0x12, 0x6a, 0x0a, 0x1a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, + 0x63, 0x61, 0x72, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x78, 0x61, 0x74, 0x75, + 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, + 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x48, 0x00, 0x52, 0x25, 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, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, + 0x52, 0x12, 0x70, 0x0a, 0x1f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x5f, + 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x78, 0x61, 0x74, + 0x75, 0x2e, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x2e, 0x42, 0x6c, 0x6f, + 0x63, 0x6b, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x48, 0x00, 0x52, 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x50, 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, + 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, 0x53, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, + 0x49, 0x4f, 0x4e, 0x12, 0x6b, 0x0a, 0x20, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, + 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, + 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, + 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x48, 0x00, 0x52, 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, + 0x12, 0x54, 0x0a, 0x16, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x70, 0x32, 0x70, 0x5f, 0x61, + 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x25, 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, 0x16, + 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x50, 0x32, 0x50, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, + 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x5a, 0x0a, 0x14, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, + 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x5f, 0x64, 0x75, 0x74, 0x79, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, - 0x76, 0x31, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x56, 0x32, 0x48, - 0x00, 0x52, 0x29, 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, 0x57, 0x49, 0x54, 0x48, 0x44, 0x52, 0x41, 0x57, 0x41, 0x4c, 0x12, 0x6a, 0x0a, 0x1a, - 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x5f, 0x62, 0x6c, - 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1d, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x48, - 0x00, 0x52, 0x25, 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, 0x42, - 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x70, 0x0a, 0x1f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x70, 0x72, 0x69, 0x6e, 0x74, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x63, 0x6c, 0x61, - 0x73, 0x73, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x23, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x24, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x70, 0x72, - 0x69, 0x6e, 0x74, 0x2e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x43, 0x6c, 0x61, 0x73, 0x73, 0x69, 0x66, - 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x1f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, - 0x50, 0x52, 0x49, 0x4e, 0x54, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x43, 0x4c, 0x41, 0x53, - 0x53, 0x49, 0x46, 0x49, 0x43, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x6b, 0x0a, 0x20, 0x65, 0x74, - 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x24, + 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, 0x44, 0x75, 0x74, 0x79, 0x48, + 0x00, 0x52, 0x1f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, + 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x50, 0x52, 0x4f, 0x50, 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, + 0x54, 0x59, 0x12, 0x8f, 0x01, 0x0a, 0x2a, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, + 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x65, 0x6c, 0x61, 0x62, 0x6f, + 0x72, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, + 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, + 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x35, 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, 0x4c, + 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x4c, 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, + 0x72, 0x61, 0x63, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x28, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, + 0x70, 0x2e, 0x41, 0x64, 0x64, 0x50, 0x65, 0x65, 0x72, 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, + 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x45, + 0x45, 0x52, 0x12, 0x55, 0x0a, 0x18, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, + 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x29, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x50, 0x65, 0x65, 0x72, 0x48, 0x00, 0x52, + 0x18, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, + 0x4d, 0x4f, 0x56, 0x45, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x12, 0x4c, 0x0a, 0x15, 0x6c, 0x69, 0x62, + 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x76, 0x5f, 0x72, + 0x70, 0x63, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x52, 0x65, 0x63, 0x76, 0x52, 0x50, 0x43, 0x48, 0x00, + 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, + 0x45, 0x43, 0x56, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x4c, 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, + 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x73, 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x70, 0x63, + 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, + 0x62, 0x70, 0x32, 0x70, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x52, 0x50, 0x43, 0x48, 0x00, 0x52, 0x15, + 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x53, 0x45, 0x4e, + 0x44, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x41, 0x0a, 0x11, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, + 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x6a, 0x6f, 0x69, 0x6e, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x4a, + 0x6f, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x11, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, + 0x41, 0x43, 0x45, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x12, 0x50, 0x0a, 0x16, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x65, 0x64, 0x18, 0x2d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, + 0x48, 0x00, 0x52, 0x16, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, + 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, 0x44, 0x12, 0x59, 0x0a, 0x19, 0x6c, 0x69, + 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x64, 0x69, 0x73, 0x63, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x2e, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x44, 0x69, 0x73, 0x63, + 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x19, 0x4c, 0x49, 0x42, 0x50, + 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x43, 0x4f, 0x4e, 0x4e, + 0x45, 0x43, 0x54, 0x45, 0x44, 0x12, 0x61, 0x0a, 0x1c, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, + 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x65, 0x74, + 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x2f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x78, 0x61, + 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x1c, 0x4c, 0x49, 0x42, 0x50, + 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x5f, + 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x12, 0x5b, 0x0a, 0x1a, 0x6c, 0x69, 0x62, 0x70, + 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x5f, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x30, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, + 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x6c, + 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, 0x52, 0x1a, 0x4c, 0x49, 0x42, 0x50, 0x32, + 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x53, + 0x54, 0x41, 0x54, 0x55, 0x53, 0x12, 0x7a, 0x0a, 0x23, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, + 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, + 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x31, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, + 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x42, + 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, + 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, + 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, + 0x4b, 0x12, 0x78, 0x0a, 0x29, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, + 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x65, 0x61, 0x63, + 0x6f, 0x6e, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x32, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, - 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x48, 0x00, - 0x52, 0x25, 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, 0x42, 0x4c, 0x4f, 0x42, 0x5f, - 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x54, 0x0a, 0x16, 0x62, 0x65, 0x61, 0x63, 0x6f, - 0x6e, 0x5f, 0x70, 0x32, 0x70, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x25, 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, 0x16, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x50, 0x32, - 0x50, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x5a, 0x0a, - 0x14, 0x65, 0x74, 0x68, 0x5f, 0x76, 0x31, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x6f, 0x73, 0x65, 0x72, - 0x5f, 0x64, 0x75, 0x74, 0x79, 0x18, 0x26, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x6f, 0x73, - 0x65, 0x72, 0x44, 0x75, 0x74, 0x79, 0x48, 0x00, 0x52, 0x1f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, - 0x5f, 0x41, 0x50, 0x49, 0x5f, 0x45, 0x54, 0x48, 0x5f, 0x56, 0x31, 0x5f, 0x50, 0x52, 0x4f, 0x50, - 0x4f, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x55, 0x54, 0x59, 0x12, 0x8f, 0x01, 0x0a, 0x2a, 0x65, 0x74, - 0x68, 0x5f, 0x76, 0x32, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, - 0x6b, 0x5f, 0x65, 0x6c, 0x61, 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x74, - 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x27, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, - 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x61, - 0x62, 0x6f, 0x72, 0x61, 0x74, 0x65, 0x64, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x35, 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, 0x4c, 0x41, 0x42, 0x4f, 0x52, 0x41, 0x54, 0x45, 0x44, 0x5f, - 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x4c, 0x0a, 0x15, 0x6c, - 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x5f, - 0x70, 0x65, 0x65, 0x72, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x78, 0x61, 0x74, - 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x41, 0x64, 0x64, 0x50, 0x65, 0x65, 0x72, - 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, - 0x5f, 0x41, 0x44, 0x44, 0x5f, 0x50, 0x45, 0x45, 0x52, 0x12, 0x55, 0x0a, 0x18, 0x6c, 0x69, 0x62, - 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x6d, 0x6f, 0x76, 0x65, - 0x5f, 0x70, 0x65, 0x65, 0x72, 0x18, 0x29, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x78, 0x61, - 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, - 0x50, 0x65, 0x65, 0x72, 0x48, 0x00, 0x52, 0x18, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, - 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, 0x4d, 0x4f, 0x56, 0x45, 0x5f, 0x50, 0x45, 0x45, 0x52, - 0x12, 0x4c, 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, - 0x5f, 0x72, 0x65, 0x63, 0x76, 0x5f, 0x72, 0x70, 0x63, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x14, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x52, 0x65, - 0x63, 0x76, 0x52, 0x50, 0x43, 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, - 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x52, 0x45, 0x43, 0x56, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x4c, - 0x0a, 0x15, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x73, - 0x65, 0x6e, 0x64, 0x5f, 0x72, 0x70, 0x63, 0x18, 0x2b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, - 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x53, 0x65, 0x6e, 0x64, - 0x52, 0x50, 0x43, 0x48, 0x00, 0x52, 0x15, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, - 0x41, 0x43, 0x45, 0x5f, 0x53, 0x45, 0x4e, 0x44, 0x5f, 0x52, 0x50, 0x43, 0x12, 0x41, 0x0a, 0x11, - 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x6a, 0x6f, 0x69, - 0x6e, 0x18, 0x2c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, - 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x4a, 0x6f, 0x69, 0x6e, 0x48, 0x00, 0x52, 0x11, 0x4c, 0x49, - 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x4a, 0x4f, 0x49, 0x4e, 0x12, - 0x50, 0x0a, 0x16, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, - 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x2d, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x16, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x43, 0x6f, - 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x16, 0x4c, 0x49, 0x42, 0x50, 0x32, - 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, - 0x44, 0x12, 0x59, 0x0a, 0x19, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, - 0x65, 0x5f, 0x64, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x18, 0x2e, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, - 0x32, 0x70, 0x2e, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x48, - 0x00, 0x52, 0x19, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, - 0x44, 0x49, 0x53, 0x43, 0x4f, 0x4e, 0x4e, 0x45, 0x43, 0x54, 0x45, 0x44, 0x12, 0x61, 0x0a, 0x1c, - 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x68, 0x61, 0x6e, - 0x64, 0x6c, 0x65, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x2f, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, - 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, - 0x00, 0x52, 0x1c, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, - 0x48, 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x44, 0x41, 0x54, 0x41, 0x12, - 0x5b, 0x0a, 0x1a, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, - 0x68, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x30, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, - 0x70, 0x2e, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, - 0x52, 0x1a, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x48, - 0x41, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55, 0x53, 0x12, 0x7a, 0x0a, 0x23, - 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, - 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, - 0x6f, 0x63, 0x6b, 0x18, 0x31, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x78, 0x61, 0x74, 0x75, - 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, - 0x62, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42, 0x6c, 0x6f, 0x63, - 0x6b, 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, - 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, 0x43, - 0x4f, 0x4e, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x12, 0x78, 0x0a, 0x29, 0x6c, 0x69, 0x62, 0x70, - 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, - 0x75, 0x62, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x61, 0x74, 0x74, 0x65, 0x73, 0x74, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x32, 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, 0x48, 0x00, 0x52, 0x29, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, - 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, - 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, - 0x4f, 0x4e, 0x12, 0x7a, 0x0a, 0x23, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, - 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x6c, 0x6f, - 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x26, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x67, 0x6f, - 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x42, 0x6c, 0x6f, 0x62, - 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, - 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, - 0x42, 0x5f, 0x42, 0x4c, 0x4f, 0x42, 0x5f, 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 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, + 0x76, 0x31, 0x2e, 0x41, 0x74, 0x74, 0x65, 0x73, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, + 0x52, 0x29, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, 0x5f, 0x47, + 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x45, 0x41, 0x43, 0x4f, 0x4e, 0x5f, + 0x41, 0x54, 0x54, 0x45, 0x53, 0x54, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x12, 0x7a, 0x0a, 0x23, 0x6c, + 0x69, 0x62, 0x70, 0x32, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x63, 0x65, 0x5f, 0x67, 0x6f, 0x73, 0x73, + 0x69, 0x70, 0x73, 0x75, 0x62, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x63, + 0x61, 0x72, 0x18, 0x33, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, + 0x6c, 0x69, 0x62, 0x70, 0x32, 0x70, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x73, 0x75, 0x62, + 0x2e, 0x65, 0x74, 0x68, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x53, 0x69, 0x64, 0x65, 0x63, 0x61, 0x72, + 0x48, 0x00, 0x52, 0x23, 0x4c, 0x49, 0x42, 0x50, 0x32, 0x50, 0x5f, 0x54, 0x52, 0x41, 0x43, 0x45, + 0x5f, 0x47, 0x4f, 0x53, 0x53, 0x49, 0x50, 0x53, 0x55, 0x42, 0x5f, 0x42, 0x4c, 0x4f, 0x42, 0x5f, + 0x53, 0x49, 0x44, 0x45, 0x43, 0x41, 0x52, 0x12, 0x52, 0x0a, 0x11, 0x65, 0x74, 0x68, 0x5f, 0x76, + 0x31, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x34, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x78, 0x61, 0x74, 0x75, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x6f, 0x72, 0x73, 0x48, 0x00, 0x52, 0x23, 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, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x41, 0x54, 0x4f, 0x52, 0x53, 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 ( @@ -9641,7 +9794,7 @@ func file_pkg_proto_xatu_event_ingester_proto_rawDescGZIP() []byte { } var file_pkg_proto_xatu_event_ingester_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_pkg_proto_xatu_event_ingester_proto_msgTypes = make([]protoimpl.MessageInfo, 91) +var file_pkg_proto_xatu_event_ingester_proto_msgTypes = make([]protoimpl.MessageInfo, 93) var file_pkg_proto_xatu_event_ingester_proto_goTypes = []interface{}{ (Event_Name)(0), // 0: xatu.Event.Name (*CreateEventsRequest)(nil), // 1: xatu.CreateEventsRequest @@ -9657,438 +9810,445 @@ var file_pkg_proto_xatu_event_ingester_proto_goTypes = []interface{}{ (*AttestingValidatorV2)(nil), // 11: xatu.AttestingValidatorV2 (*DebugForkChoiceReorg)(nil), // 12: xatu.DebugForkChoiceReorg (*DebugForkChoiceReorgV2)(nil), // 13: xatu.DebugForkChoiceReorgV2 - (*BlockIdentifier)(nil), // 14: xatu.BlockIdentifier - (*ClientMeta)(nil), // 15: xatu.ClientMeta - (*ServerMeta)(nil), // 16: xatu.ServerMeta - (*Meta)(nil), // 17: xatu.Meta - (*Event)(nil), // 18: xatu.Event - (*DecoratedEvent)(nil), // 19: xatu.DecoratedEvent - (*ClientMeta_Ethereum)(nil), // 20: xatu.ClientMeta.Ethereum - nil, // 21: xatu.ClientMeta.LabelsEntry - (*ClientMeta_AdditionalEthV1AttestationSourceData)(nil), // 22: xatu.ClientMeta.AdditionalEthV1AttestationSourceData - (*ClientMeta_AdditionalEthV1AttestationSourceV2Data)(nil), // 23: xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data - (*ClientMeta_AdditionalEthV1AttestationTargetData)(nil), // 24: xatu.ClientMeta.AdditionalEthV1AttestationTargetData - (*ClientMeta_AdditionalEthV1AttestationTargetV2Data)(nil), // 25: xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data - (*ClientMeta_AdditionalEthV1EventsAttestationData)(nil), // 26: xatu.ClientMeta.AdditionalEthV1EventsAttestationData - (*ClientMeta_AdditionalEthV1EventsAttestationV2Data)(nil), // 27: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data - (*ClientMeta_AdditionalEthV1EventsHeadData)(nil), // 28: xatu.ClientMeta.AdditionalEthV1EventsHeadData - (*ClientMeta_AdditionalEthV1EventsHeadV2Data)(nil), // 29: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data - (*ClientMeta_AdditionalEthV1EventsBlockData)(nil), // 30: xatu.ClientMeta.AdditionalEthV1EventsBlockData - (*ClientMeta_AdditionalEthV1EventsBlockV2Data)(nil), // 31: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data - (*ClientMeta_AdditionalEthV1EventsVoluntaryExitData)(nil), // 32: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitData - (*ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data)(nil), // 33: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitV2Data - (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData)(nil), // 34: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointData - (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data)(nil), // 35: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointV2Data - (*ClientMeta_AdditionalEthV1EventsChainReorgData)(nil), // 36: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData - (*ClientMeta_AdditionalEthV1EventsChainReorgV2Data)(nil), // 37: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data - (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData)(nil), // 38: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData - (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data)(nil), // 39: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data - (*ClientMeta_AdditionalEthV1EventsContributionAndProofData)(nil), // 40: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofData - (*ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data)(nil), // 41: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofV2Data - (*ClientMeta_ForkChoiceSnapshot)(nil), // 42: xatu.ClientMeta.ForkChoiceSnapshot - (*ClientMeta_ForkChoiceSnapshotV2)(nil), // 43: xatu.ClientMeta.ForkChoiceSnapshotV2 - (*ClientMeta_AdditionalEthV1DebugForkChoiceData)(nil), // 44: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceData - (*ClientMeta_AdditionalEthV1DebugForkChoiceV2Data)(nil), // 45: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceV2Data - (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData)(nil), // 46: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData - (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data)(nil), // 47: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data - (*ClientMeta_AdditionalEthV1BeaconCommitteeData)(nil), // 48: xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData - (*ClientMeta_AdditionalMempoolTransactionData)(nil), // 49: xatu.ClientMeta.AdditionalMempoolTransactionData - (*ClientMeta_AdditionalMempoolTransactionV2Data)(nil), // 50: xatu.ClientMeta.AdditionalMempoolTransactionV2Data - (*ClientMeta_AdditionalEthV2BeaconBlockData)(nil), // 51: xatu.ClientMeta.AdditionalEthV2BeaconBlockData - (*ClientMeta_AdditionalEthV2BeaconBlockV2Data)(nil), // 52: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data - (*ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData)(nil), // 53: xatu.ClientMeta.AdditionalEthV2BeaconBlockAttesterSlashingData - (*ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData)(nil), // 54: xatu.ClientMeta.AdditionalEthV2BeaconBlockProposerSlashingData - (*ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData)(nil), // 55: xatu.ClientMeta.AdditionalEthV2BeaconBlockVoluntaryExitData - (*ClientMeta_AdditionalEthV2BeaconBlockDepositData)(nil), // 56: xatu.ClientMeta.AdditionalEthV2BeaconBlockDepositData - (*ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData)(nil), // 57: xatu.ClientMeta.AdditionalEthV2BeaconBlockBLSToExecutionChangeData - (*ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData)(nil), // 58: xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData - (*ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData)(nil), // 59: xatu.ClientMeta.AdditionalEthV2BeaconBlockWithdrawalData - (*ClientMeta_AdditionalBlockprintBlockClassificationData)(nil), // 60: xatu.ClientMeta.AdditionalBlockprintBlockClassificationData - (*ClientMeta_AttestationDataSnapshot)(nil), // 61: xatu.ClientMeta.AttestationDataSnapshot - (*ClientMeta_AdditionalEthV1ValidatorAttestationDataData)(nil), // 62: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData - (*ClientMeta_AdditionalEthV1EventsBlobSidecarData)(nil), // 63: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData - (*ClientMeta_AdditionalEthV1BeaconBlobSidecarData)(nil), // 64: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData - (*ClientMeta_AdditionalBeaconP2PAttestationData)(nil), // 65: xatu.ClientMeta.AdditionalBeaconP2PAttestationData - (*ClientMeta_AdditionalEthV1ProposerDutyData)(nil), // 66: xatu.ClientMeta.AdditionalEthV1ProposerDutyData - (*ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData)(nil), // 67: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData - (*ClientMeta_AdditionalLibP2PTraceAddPeerData)(nil), // 68: xatu.ClientMeta.AdditionalLibP2PTraceAddPeerData - (*ClientMeta_AdditionalLibP2PTraceRemovePeerData)(nil), // 69: xatu.ClientMeta.AdditionalLibP2PTraceRemovePeerData - (*ClientMeta_AdditionalLibP2PTraceRecvRPCData)(nil), // 70: xatu.ClientMeta.AdditionalLibP2PTraceRecvRPCData - (*ClientMeta_AdditionalLibP2PTraceSendRPCData)(nil), // 71: xatu.ClientMeta.AdditionalLibP2PTraceSendRPCData - (*ClientMeta_AdditionalLibP2PTraceJoinData)(nil), // 72: xatu.ClientMeta.AdditionalLibP2PTraceJoinData - (*ClientMeta_AdditionalLibP2PTraceConnectedData)(nil), // 73: xatu.ClientMeta.AdditionalLibP2PTraceConnectedData - (*ClientMeta_AdditionalLibP2PTraceDisconnectedData)(nil), // 74: xatu.ClientMeta.AdditionalLibP2PTraceDisconnectedData - (*ClientMeta_AdditionalLibP2PTraceHandleMetadataData)(nil), // 75: xatu.ClientMeta.AdditionalLibP2PTraceHandleMetadataData - (*ClientMeta_AdditionalLibP2PTraceHandleStatusData)(nil), // 76: xatu.ClientMeta.AdditionalLibP2PTraceHandleStatusData - (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData)(nil), // 77: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData - (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData)(nil), // 78: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData - (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData)(nil), // 79: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData - (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData)(nil), // 80: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData - (*ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData)(nil), // 81: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData - (*ClientMeta_Ethereum_Network)(nil), // 82: xatu.ClientMeta.Ethereum.Network - (*ClientMeta_Ethereum_Execution)(nil), // 83: xatu.ClientMeta.Ethereum.Execution - (*ClientMeta_Ethereum_Consensus)(nil), // 84: xatu.ClientMeta.Ethereum.Consensus - (*ServerMeta_Event)(nil), // 85: xatu.ServerMeta.Event - (*ServerMeta_Geo)(nil), // 86: xatu.ServerMeta.Geo - (*ServerMeta_Client)(nil), // 87: xatu.ServerMeta.Client - (*ServerMeta_Peer)(nil), // 88: xatu.ServerMeta.Peer - (*ServerMeta_AdditionalBeaconP2PAttestationData)(nil), // 89: xatu.ServerMeta.AdditionalBeaconP2PAttestationData - (*ServerMeta_AdditionalLibp2PTraceConnectedData)(nil), // 90: xatu.ServerMeta.AdditionalLibp2PTraceConnectedData - (*ServerMeta_AdditionalLibp2PTraceDisconnectedData)(nil), // 91: xatu.ServerMeta.AdditionalLibp2PTraceDisconnectedData - (*timestamppb.Timestamp)(nil), // 92: google.protobuf.Timestamp - (*wrapperspb.UInt64Value)(nil), // 93: google.protobuf.UInt64Value - (*v1.ForkChoice)(nil), // 94: xatu.eth.v1.ForkChoice - (*v1.EventChainReorg)(nil), // 95: xatu.eth.v1.EventChainReorg - (*v1.ForkChoiceV2)(nil), // 96: xatu.eth.v1.ForkChoiceV2 - (*v1.EventChainReorgV2)(nil), // 97: xatu.eth.v1.EventChainReorgV2 - (*v1.Attestation)(nil), // 98: xatu.eth.v1.Attestation - (*v1.EventBlock)(nil), // 99: xatu.eth.v1.EventBlock - (*v1.EventFinalizedCheckpoint)(nil), // 100: xatu.eth.v1.EventFinalizedCheckpoint - (*v1.EventHead)(nil), // 101: xatu.eth.v1.EventHead - (*v1.EventVoluntaryExit)(nil), // 102: xatu.eth.v1.EventVoluntaryExit - (*v1.EventContributionAndProof)(nil), // 103: xatu.eth.v1.EventContributionAndProof - (*v2.EventBlock)(nil), // 104: xatu.eth.v2.EventBlock - (*v1.Committee)(nil), // 105: xatu.eth.v1.Committee - (*v1.AttestationDataV2)(nil), // 106: xatu.eth.v1.AttestationDataV2 - (*v1.AttestationV2)(nil), // 107: xatu.eth.v1.AttestationV2 - (*v1.EventBlockV2)(nil), // 108: xatu.eth.v1.EventBlockV2 - (*v1.EventFinalizedCheckpointV2)(nil), // 109: xatu.eth.v1.EventFinalizedCheckpointV2 - (*v1.EventHeadV2)(nil), // 110: xatu.eth.v1.EventHeadV2 - (*v1.EventVoluntaryExitV2)(nil), // 111: xatu.eth.v1.EventVoluntaryExitV2 - (*v1.EventContributionAndProofV2)(nil), // 112: xatu.eth.v1.EventContributionAndProofV2 - (*v2.EventBlockV2)(nil), // 113: xatu.eth.v2.EventBlockV2 - (*v1.AttesterSlashingV2)(nil), // 114: xatu.eth.v1.AttesterSlashingV2 - (*v1.ProposerSlashingV2)(nil), // 115: xatu.eth.v1.ProposerSlashingV2 - (*v1.SignedVoluntaryExitV2)(nil), // 116: xatu.eth.v1.SignedVoluntaryExitV2 - (*v1.DepositV2)(nil), // 117: xatu.eth.v1.DepositV2 - (*v2.SignedBLSToExecutionChangeV2)(nil), // 118: xatu.eth.v2.SignedBLSToExecutionChangeV2 - (*v1.Transaction)(nil), // 119: xatu.eth.v1.Transaction - (*v1.WithdrawalV2)(nil), // 120: xatu.eth.v1.WithdrawalV2 - (*v1.EventBlobSidecar)(nil), // 121: xatu.eth.v1.EventBlobSidecar - (*blockprint.BlockClassification)(nil), // 122: xatu.blockprint.BlockClassification - (*v1.BlobSidecar)(nil), // 123: xatu.eth.v1.BlobSidecar - (*v1.ProposerDuty)(nil), // 124: xatu.eth.v1.ProposerDuty - (*v1.ElaboratedAttestation)(nil), // 125: xatu.eth.v1.ElaboratedAttestation - (*libp2p.AddPeer)(nil), // 126: xatu.libp2p.AddPeer - (*libp2p.RemovePeer)(nil), // 127: xatu.libp2p.RemovePeer - (*libp2p.RecvRPC)(nil), // 128: xatu.libp2p.RecvRPC - (*libp2p.SendRPC)(nil), // 129: xatu.libp2p.SendRPC - (*libp2p.Join)(nil), // 130: xatu.libp2p.Join - (*libp2p.Connected)(nil), // 131: xatu.libp2p.Connected - (*libp2p.Disconnected)(nil), // 132: xatu.libp2p.Disconnected - (*libp2p.HandleMetadata)(nil), // 133: xatu.libp2p.HandleMetadata - (*libp2p.HandleStatus)(nil), // 134: xatu.libp2p.HandleStatus - (*gossipsub.BeaconBlock)(nil), // 135: xatu.libp2p.gossipsub.eth.BeaconBlock - (*gossipsub.BlobSidecar)(nil), // 136: xatu.libp2p.gossipsub.eth.BlobSidecar - (*wrapperspb.UInt32Value)(nil), // 137: google.protobuf.UInt32Value - (*libp2p.Peer)(nil), // 138: xatu.libp2p.Peer - (*wrapperspb.BoolValue)(nil), // 139: google.protobuf.BoolValue - (*libp2p.TraceEventMetadata)(nil), // 140: xatu.libp2p.TraceEventMetadata - (*wrapperspb.StringValue)(nil), // 141: google.protobuf.StringValue + (*Validators)(nil), // 14: xatu.Validators + (*BlockIdentifier)(nil), // 15: xatu.BlockIdentifier + (*ClientMeta)(nil), // 16: xatu.ClientMeta + (*ServerMeta)(nil), // 17: xatu.ServerMeta + (*Meta)(nil), // 18: xatu.Meta + (*Event)(nil), // 19: xatu.Event + (*DecoratedEvent)(nil), // 20: xatu.DecoratedEvent + (*ClientMeta_Ethereum)(nil), // 21: xatu.ClientMeta.Ethereum + nil, // 22: xatu.ClientMeta.LabelsEntry + (*ClientMeta_AdditionalEthV1AttestationSourceData)(nil), // 23: xatu.ClientMeta.AdditionalEthV1AttestationSourceData + (*ClientMeta_AdditionalEthV1AttestationSourceV2Data)(nil), // 24: xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data + (*ClientMeta_AdditionalEthV1AttestationTargetData)(nil), // 25: xatu.ClientMeta.AdditionalEthV1AttestationTargetData + (*ClientMeta_AdditionalEthV1AttestationTargetV2Data)(nil), // 26: xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data + (*ClientMeta_AdditionalEthV1EventsAttestationData)(nil), // 27: xatu.ClientMeta.AdditionalEthV1EventsAttestationData + (*ClientMeta_AdditionalEthV1EventsAttestationV2Data)(nil), // 28: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data + (*ClientMeta_AdditionalEthV1EventsHeadData)(nil), // 29: xatu.ClientMeta.AdditionalEthV1EventsHeadData + (*ClientMeta_AdditionalEthV1EventsHeadV2Data)(nil), // 30: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data + (*ClientMeta_AdditionalEthV1EventsBlockData)(nil), // 31: xatu.ClientMeta.AdditionalEthV1EventsBlockData + (*ClientMeta_AdditionalEthV1EventsBlockV2Data)(nil), // 32: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data + (*ClientMeta_AdditionalEthV1EventsVoluntaryExitData)(nil), // 33: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitData + (*ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data)(nil), // 34: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitV2Data + (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData)(nil), // 35: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointData + (*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data)(nil), // 36: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointV2Data + (*ClientMeta_AdditionalEthV1EventsChainReorgData)(nil), // 37: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData + (*ClientMeta_AdditionalEthV1EventsChainReorgV2Data)(nil), // 38: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data + (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData)(nil), // 39: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData + (*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data)(nil), // 40: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data + (*ClientMeta_AdditionalEthV1EventsContributionAndProofData)(nil), // 41: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofData + (*ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data)(nil), // 42: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofV2Data + (*ClientMeta_ForkChoiceSnapshot)(nil), // 43: xatu.ClientMeta.ForkChoiceSnapshot + (*ClientMeta_ForkChoiceSnapshotV2)(nil), // 44: xatu.ClientMeta.ForkChoiceSnapshotV2 + (*ClientMeta_AdditionalEthV1DebugForkChoiceData)(nil), // 45: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceData + (*ClientMeta_AdditionalEthV1DebugForkChoiceV2Data)(nil), // 46: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceV2Data + (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData)(nil), // 47: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData + (*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data)(nil), // 48: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data + (*ClientMeta_AdditionalEthV1BeaconCommitteeData)(nil), // 49: xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData + (*ClientMeta_AdditionalMempoolTransactionData)(nil), // 50: xatu.ClientMeta.AdditionalMempoolTransactionData + (*ClientMeta_AdditionalMempoolTransactionV2Data)(nil), // 51: xatu.ClientMeta.AdditionalMempoolTransactionV2Data + (*ClientMeta_AdditionalEthV2BeaconBlockData)(nil), // 52: xatu.ClientMeta.AdditionalEthV2BeaconBlockData + (*ClientMeta_AdditionalEthV2BeaconBlockV2Data)(nil), // 53: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data + (*ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData)(nil), // 54: xatu.ClientMeta.AdditionalEthV2BeaconBlockAttesterSlashingData + (*ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData)(nil), // 55: xatu.ClientMeta.AdditionalEthV2BeaconBlockProposerSlashingData + (*ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData)(nil), // 56: xatu.ClientMeta.AdditionalEthV2BeaconBlockVoluntaryExitData + (*ClientMeta_AdditionalEthV2BeaconBlockDepositData)(nil), // 57: xatu.ClientMeta.AdditionalEthV2BeaconBlockDepositData + (*ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData)(nil), // 58: xatu.ClientMeta.AdditionalEthV2BeaconBlockBLSToExecutionChangeData + (*ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData)(nil), // 59: xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData + (*ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData)(nil), // 60: xatu.ClientMeta.AdditionalEthV2BeaconBlockWithdrawalData + (*ClientMeta_AdditionalBlockprintBlockClassificationData)(nil), // 61: xatu.ClientMeta.AdditionalBlockprintBlockClassificationData + (*ClientMeta_AttestationDataSnapshot)(nil), // 62: xatu.ClientMeta.AttestationDataSnapshot + (*ClientMeta_AdditionalEthV1ValidatorAttestationDataData)(nil), // 63: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData + (*ClientMeta_AdditionalEthV1EventsBlobSidecarData)(nil), // 64: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData + (*ClientMeta_AdditionalEthV1BeaconBlobSidecarData)(nil), // 65: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData + (*ClientMeta_AdditionalBeaconP2PAttestationData)(nil), // 66: xatu.ClientMeta.AdditionalBeaconP2PAttestationData + (*ClientMeta_AdditionalEthV1ProposerDutyData)(nil), // 67: xatu.ClientMeta.AdditionalEthV1ProposerDutyData + (*ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData)(nil), // 68: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData + (*ClientMeta_AdditionalLibP2PTraceAddPeerData)(nil), // 69: xatu.ClientMeta.AdditionalLibP2PTraceAddPeerData + (*ClientMeta_AdditionalLibP2PTraceRemovePeerData)(nil), // 70: xatu.ClientMeta.AdditionalLibP2PTraceRemovePeerData + (*ClientMeta_AdditionalLibP2PTraceRecvRPCData)(nil), // 71: xatu.ClientMeta.AdditionalLibP2PTraceRecvRPCData + (*ClientMeta_AdditionalLibP2PTraceSendRPCData)(nil), // 72: xatu.ClientMeta.AdditionalLibP2PTraceSendRPCData + (*ClientMeta_AdditionalLibP2PTraceJoinData)(nil), // 73: xatu.ClientMeta.AdditionalLibP2PTraceJoinData + (*ClientMeta_AdditionalLibP2PTraceConnectedData)(nil), // 74: xatu.ClientMeta.AdditionalLibP2PTraceConnectedData + (*ClientMeta_AdditionalLibP2PTraceDisconnectedData)(nil), // 75: xatu.ClientMeta.AdditionalLibP2PTraceDisconnectedData + (*ClientMeta_AdditionalLibP2PTraceHandleMetadataData)(nil), // 76: xatu.ClientMeta.AdditionalLibP2PTraceHandleMetadataData + (*ClientMeta_AdditionalLibP2PTraceHandleStatusData)(nil), // 77: xatu.ClientMeta.AdditionalLibP2PTraceHandleStatusData + (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData)(nil), // 78: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData + (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData)(nil), // 79: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData + (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData)(nil), // 80: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData + (*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData)(nil), // 81: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData + (*ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData)(nil), // 82: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData + (*ClientMeta_AdditionalEthV1ValidatorsData)(nil), // 83: xatu.ClientMeta.AdditionalEthV1ValidatorsData + (*ClientMeta_Ethereum_Network)(nil), // 84: xatu.ClientMeta.Ethereum.Network + (*ClientMeta_Ethereum_Execution)(nil), // 85: xatu.ClientMeta.Ethereum.Execution + (*ClientMeta_Ethereum_Consensus)(nil), // 86: xatu.ClientMeta.Ethereum.Consensus + (*ServerMeta_Event)(nil), // 87: xatu.ServerMeta.Event + (*ServerMeta_Geo)(nil), // 88: xatu.ServerMeta.Geo + (*ServerMeta_Client)(nil), // 89: xatu.ServerMeta.Client + (*ServerMeta_Peer)(nil), // 90: xatu.ServerMeta.Peer + (*ServerMeta_AdditionalBeaconP2PAttestationData)(nil), // 91: xatu.ServerMeta.AdditionalBeaconP2PAttestationData + (*ServerMeta_AdditionalLibp2PTraceConnectedData)(nil), // 92: xatu.ServerMeta.AdditionalLibp2PTraceConnectedData + (*ServerMeta_AdditionalLibp2PTraceDisconnectedData)(nil), // 93: xatu.ServerMeta.AdditionalLibp2PTraceDisconnectedData + (*timestamppb.Timestamp)(nil), // 94: google.protobuf.Timestamp + (*wrapperspb.UInt64Value)(nil), // 95: google.protobuf.UInt64Value + (*v1.ForkChoice)(nil), // 96: xatu.eth.v1.ForkChoice + (*v1.EventChainReorg)(nil), // 97: xatu.eth.v1.EventChainReorg + (*v1.ForkChoiceV2)(nil), // 98: xatu.eth.v1.ForkChoiceV2 + (*v1.EventChainReorgV2)(nil), // 99: xatu.eth.v1.EventChainReorgV2 + (*v1.Validator)(nil), // 100: xatu.eth.v1.Validator + (*v1.Attestation)(nil), // 101: xatu.eth.v1.Attestation + (*v1.EventBlock)(nil), // 102: xatu.eth.v1.EventBlock + (*v1.EventFinalizedCheckpoint)(nil), // 103: xatu.eth.v1.EventFinalizedCheckpoint + (*v1.EventHead)(nil), // 104: xatu.eth.v1.EventHead + (*v1.EventVoluntaryExit)(nil), // 105: xatu.eth.v1.EventVoluntaryExit + (*v1.EventContributionAndProof)(nil), // 106: xatu.eth.v1.EventContributionAndProof + (*v2.EventBlock)(nil), // 107: xatu.eth.v2.EventBlock + (*v1.Committee)(nil), // 108: xatu.eth.v1.Committee + (*v1.AttestationDataV2)(nil), // 109: xatu.eth.v1.AttestationDataV2 + (*v1.AttestationV2)(nil), // 110: xatu.eth.v1.AttestationV2 + (*v1.EventBlockV2)(nil), // 111: xatu.eth.v1.EventBlockV2 + (*v1.EventFinalizedCheckpointV2)(nil), // 112: xatu.eth.v1.EventFinalizedCheckpointV2 + (*v1.EventHeadV2)(nil), // 113: xatu.eth.v1.EventHeadV2 + (*v1.EventVoluntaryExitV2)(nil), // 114: xatu.eth.v1.EventVoluntaryExitV2 + (*v1.EventContributionAndProofV2)(nil), // 115: xatu.eth.v1.EventContributionAndProofV2 + (*v2.EventBlockV2)(nil), // 116: xatu.eth.v2.EventBlockV2 + (*v1.AttesterSlashingV2)(nil), // 117: xatu.eth.v1.AttesterSlashingV2 + (*v1.ProposerSlashingV2)(nil), // 118: xatu.eth.v1.ProposerSlashingV2 + (*v1.SignedVoluntaryExitV2)(nil), // 119: xatu.eth.v1.SignedVoluntaryExitV2 + (*v1.DepositV2)(nil), // 120: xatu.eth.v1.DepositV2 + (*v2.SignedBLSToExecutionChangeV2)(nil), // 121: xatu.eth.v2.SignedBLSToExecutionChangeV2 + (*v1.Transaction)(nil), // 122: xatu.eth.v1.Transaction + (*v1.WithdrawalV2)(nil), // 123: xatu.eth.v1.WithdrawalV2 + (*v1.EventBlobSidecar)(nil), // 124: xatu.eth.v1.EventBlobSidecar + (*blockprint.BlockClassification)(nil), // 125: xatu.blockprint.BlockClassification + (*v1.BlobSidecar)(nil), // 126: xatu.eth.v1.BlobSidecar + (*v1.ProposerDuty)(nil), // 127: xatu.eth.v1.ProposerDuty + (*v1.ElaboratedAttestation)(nil), // 128: xatu.eth.v1.ElaboratedAttestation + (*libp2p.AddPeer)(nil), // 129: xatu.libp2p.AddPeer + (*libp2p.RemovePeer)(nil), // 130: xatu.libp2p.RemovePeer + (*libp2p.RecvRPC)(nil), // 131: xatu.libp2p.RecvRPC + (*libp2p.SendRPC)(nil), // 132: xatu.libp2p.SendRPC + (*libp2p.Join)(nil), // 133: xatu.libp2p.Join + (*libp2p.Connected)(nil), // 134: xatu.libp2p.Connected + (*libp2p.Disconnected)(nil), // 135: xatu.libp2p.Disconnected + (*libp2p.HandleMetadata)(nil), // 136: xatu.libp2p.HandleMetadata + (*libp2p.HandleStatus)(nil), // 137: xatu.libp2p.HandleStatus + (*gossipsub.BeaconBlock)(nil), // 138: xatu.libp2p.gossipsub.eth.BeaconBlock + (*gossipsub.BlobSidecar)(nil), // 139: xatu.libp2p.gossipsub.eth.BlobSidecar + (*wrapperspb.UInt32Value)(nil), // 140: google.protobuf.UInt32Value + (*libp2p.Peer)(nil), // 141: xatu.libp2p.Peer + (*wrapperspb.BoolValue)(nil), // 142: google.protobuf.BoolValue + (*libp2p.TraceEventMetadata)(nil), // 143: xatu.libp2p.TraceEventMetadata + (*wrapperspb.StringValue)(nil), // 144: google.protobuf.StringValue } var file_pkg_proto_xatu_event_ingester_proto_depIdxs = []int32{ - 19, // 0: xatu.CreateEventsRequest.events:type_name -> xatu.DecoratedEvent - 92, // 1: xatu.Epoch.start_date_time:type_name -> google.protobuf.Timestamp - 93, // 2: xatu.EpochV2.number:type_name -> google.protobuf.UInt64Value - 92, // 3: xatu.EpochV2.start_date_time:type_name -> google.protobuf.Timestamp - 92, // 4: xatu.Slot.start_date_time:type_name -> google.protobuf.Timestamp - 93, // 5: xatu.SlotV2.number:type_name -> google.protobuf.UInt64Value - 92, // 6: xatu.SlotV2.start_date_time:type_name -> google.protobuf.Timestamp - 93, // 7: xatu.PropagationV2.slot_start_diff:type_name -> google.protobuf.UInt64Value - 93, // 8: xatu.AttestingValidatorV2.committee_index:type_name -> google.protobuf.UInt64Value - 93, // 9: xatu.AttestingValidatorV2.index:type_name -> google.protobuf.UInt64Value - 94, // 10: xatu.DebugForkChoiceReorg.before:type_name -> xatu.eth.v1.ForkChoice - 94, // 11: xatu.DebugForkChoiceReorg.after:type_name -> xatu.eth.v1.ForkChoice - 95, // 12: xatu.DebugForkChoiceReorg.event:type_name -> xatu.eth.v1.EventChainReorg - 96, // 13: xatu.DebugForkChoiceReorgV2.before:type_name -> xatu.eth.v1.ForkChoiceV2 - 96, // 14: xatu.DebugForkChoiceReorgV2.after:type_name -> xatu.eth.v1.ForkChoiceV2 - 97, // 15: xatu.DebugForkChoiceReorgV2.event:type_name -> xatu.eth.v1.EventChainReorgV2 - 4, // 16: xatu.BlockIdentifier.epoch:type_name -> xatu.EpochV2 - 6, // 17: xatu.BlockIdentifier.slot:type_name -> xatu.SlotV2 - 20, // 18: xatu.ClientMeta.ethereum:type_name -> xatu.ClientMeta.Ethereum - 21, // 19: xatu.ClientMeta.labels:type_name -> xatu.ClientMeta.LabelsEntry - 26, // 20: xatu.ClientMeta.eth_v1_events_attestation:type_name -> xatu.ClientMeta.AdditionalEthV1EventsAttestationData - 28, // 21: xatu.ClientMeta.eth_v1_events_head:type_name -> xatu.ClientMeta.AdditionalEthV1EventsHeadData - 30, // 22: xatu.ClientMeta.eth_v1_events_block:type_name -> xatu.ClientMeta.AdditionalEthV1EventsBlockData - 32, // 23: xatu.ClientMeta.eth_v1_events_voluntary_exit:type_name -> xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitData - 34, // 24: xatu.ClientMeta.eth_v1_events_finalized_checkpoint:type_name -> xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointData - 36, // 25: xatu.ClientMeta.eth_v1_events_chain_reorg:type_name -> xatu.ClientMeta.AdditionalEthV1EventsChainReorgData - 40, // 26: xatu.ClientMeta.eth_v1_events_contribution_and_proof:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofData - 49, // 27: xatu.ClientMeta.mempool_transaction:type_name -> xatu.ClientMeta.AdditionalMempoolTransactionData - 51, // 28: xatu.ClientMeta.eth_v2_beacon_block:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockData - 44, // 29: xatu.ClientMeta.eth_v1_debug_fork_choice:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceData - 46, // 30: xatu.ClientMeta.eth_v1_debug_fork_choice_reorg:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData - 48, // 31: xatu.ClientMeta.eth_v1_beacon_committee:type_name -> xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData - 62, // 32: xatu.ClientMeta.eth_v1_validator_attestation_data:type_name -> xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData - 27, // 33: xatu.ClientMeta.eth_v1_events_attestation_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data - 29, // 34: xatu.ClientMeta.eth_v1_events_head_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data - 31, // 35: xatu.ClientMeta.eth_v1_events_block_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data - 33, // 36: xatu.ClientMeta.eth_v1_events_voluntary_exit_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitV2Data - 35, // 37: xatu.ClientMeta.eth_v1_events_finalized_checkpoint_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointV2Data - 37, // 38: xatu.ClientMeta.eth_v1_events_chain_reorg_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data - 41, // 39: xatu.ClientMeta.eth_v1_events_contribution_and_proof_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofV2Data - 50, // 40: xatu.ClientMeta.mempool_transaction_v2:type_name -> xatu.ClientMeta.AdditionalMempoolTransactionV2Data - 52, // 41: xatu.ClientMeta.eth_v2_beacon_block_v2:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data - 45, // 42: xatu.ClientMeta.eth_v1_debug_fork_choice_v2:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceV2Data - 47, // 43: xatu.ClientMeta.eth_v1_debug_fork_choice_reorg_v2:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data - 53, // 44: xatu.ClientMeta.eth_v2_beacon_block_attester_slashing:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockAttesterSlashingData - 54, // 45: xatu.ClientMeta.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockProposerSlashingData - 55, // 46: xatu.ClientMeta.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockVoluntaryExitData - 56, // 47: xatu.ClientMeta.eth_v2_beacon_block_deposit:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockDepositData - 57, // 48: xatu.ClientMeta.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockBLSToExecutionChangeData - 58, // 49: xatu.ClientMeta.eth_v2_beacon_block_execution_transaction:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData - 59, // 50: xatu.ClientMeta.eth_v2_beacon_block_withdrawal:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockWithdrawalData - 63, // 51: xatu.ClientMeta.eth_v1_events_blob_sidecar:type_name -> xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData - 60, // 52: xatu.ClientMeta.blockprint_block_classification:type_name -> xatu.ClientMeta.AdditionalBlockprintBlockClassificationData - 64, // 53: xatu.ClientMeta.eth_v1_beacon_blob_sidecar:type_name -> xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData - 65, // 54: xatu.ClientMeta.beacon_p2p_attestation:type_name -> xatu.ClientMeta.AdditionalBeaconP2PAttestationData - 66, // 55: xatu.ClientMeta.eth_v1_proposer_duty:type_name -> xatu.ClientMeta.AdditionalEthV1ProposerDutyData - 67, // 56: xatu.ClientMeta.eth_v2_beacon_block_elaborated_attestation:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData - 68, // 57: xatu.ClientMeta.libp2p_trace_add_peer:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceAddPeerData - 69, // 58: xatu.ClientMeta.libp2p_trace_remove_peer:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceRemovePeerData - 70, // 59: xatu.ClientMeta.libp2p_trace_recv_rpc:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceRecvRPCData - 71, // 60: xatu.ClientMeta.libp2p_trace_send_rpc:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceSendRPCData - 72, // 61: xatu.ClientMeta.libp2p_trace_join:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceJoinData - 73, // 62: xatu.ClientMeta.libp2p_trace_connected:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceConnectedData - 74, // 63: xatu.ClientMeta.libp2p_trace_disconnected:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceDisconnectedData - 75, // 64: xatu.ClientMeta.libp2p_trace_handle_metadata:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceHandleMetadataData - 76, // 65: xatu.ClientMeta.libp2p_trace_handle_status:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceHandleStatusData - 77, // 66: xatu.ClientMeta.libp2p_trace_gossipsub_beacon_block:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData - 80, // 67: xatu.ClientMeta.libp2p_trace_gossipsub_beacon_attestation:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData - 81, // 68: xatu.ClientMeta.libp2p_trace_gossipsub_blob_sidecar:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData - 85, // 69: xatu.ServerMeta.event:type_name -> xatu.ServerMeta.Event - 87, // 70: xatu.ServerMeta.client:type_name -> xatu.ServerMeta.Client - 89, // 71: xatu.ServerMeta.BEACON_P2P_ATTESTATION:type_name -> xatu.ServerMeta.AdditionalBeaconP2PAttestationData - 90, // 72: xatu.ServerMeta.LIBP2P_TRACE_CONNECTED:type_name -> xatu.ServerMeta.AdditionalLibp2PTraceConnectedData - 91, // 73: xatu.ServerMeta.LIBP2P_TRACE_DISCONNECTED:type_name -> xatu.ServerMeta.AdditionalLibp2PTraceDisconnectedData - 15, // 74: xatu.Meta.client:type_name -> xatu.ClientMeta - 16, // 75: xatu.Meta.server:type_name -> xatu.ServerMeta - 0, // 76: xatu.Event.name:type_name -> xatu.Event.Name - 92, // 77: xatu.Event.date_time:type_name -> google.protobuf.Timestamp - 18, // 78: xatu.DecoratedEvent.event:type_name -> xatu.Event - 17, // 79: xatu.DecoratedEvent.meta:type_name -> xatu.Meta - 98, // 80: xatu.DecoratedEvent.eth_v1_events_attestation:type_name -> xatu.eth.v1.Attestation - 99, // 81: xatu.DecoratedEvent.eth_v1_events_block:type_name -> xatu.eth.v1.EventBlock - 95, // 82: xatu.DecoratedEvent.eth_v1_events_chain_reorg:type_name -> xatu.eth.v1.EventChainReorg - 100, // 83: xatu.DecoratedEvent.eth_v1_events_finalized_checkpoint:type_name -> xatu.eth.v1.EventFinalizedCheckpoint - 101, // 84: xatu.DecoratedEvent.eth_v1_events_head:type_name -> xatu.eth.v1.EventHead - 102, // 85: xatu.DecoratedEvent.eth_v1_events_voluntary_exit:type_name -> xatu.eth.v1.EventVoluntaryExit - 103, // 86: xatu.DecoratedEvent.eth_v1_events_contribution_and_proof:type_name -> xatu.eth.v1.EventContributionAndProof - 104, // 87: xatu.DecoratedEvent.eth_v2_beacon_block:type_name -> xatu.eth.v2.EventBlock - 94, // 88: xatu.DecoratedEvent.eth_v1_fork_choice:type_name -> xatu.eth.v1.ForkChoice - 12, // 89: xatu.DecoratedEvent.eth_v1_fork_choice_reorg:type_name -> xatu.DebugForkChoiceReorg - 105, // 90: xatu.DecoratedEvent.eth_v1_beacon_committee:type_name -> xatu.eth.v1.Committee - 106, // 91: xatu.DecoratedEvent.eth_v1_validator_attestation_data:type_name -> xatu.eth.v1.AttestationDataV2 - 107, // 92: xatu.DecoratedEvent.eth_v1_events_attestation_v2:type_name -> xatu.eth.v1.AttestationV2 - 108, // 93: xatu.DecoratedEvent.eth_v1_events_block_v2:type_name -> xatu.eth.v1.EventBlockV2 - 97, // 94: xatu.DecoratedEvent.eth_v1_events_chain_reorg_v2:type_name -> xatu.eth.v1.EventChainReorgV2 - 109, // 95: xatu.DecoratedEvent.eth_v1_events_finalized_checkpoint_v2:type_name -> xatu.eth.v1.EventFinalizedCheckpointV2 - 110, // 96: xatu.DecoratedEvent.eth_v1_events_head_v2:type_name -> xatu.eth.v1.EventHeadV2 - 111, // 97: xatu.DecoratedEvent.eth_v1_events_voluntary_exit_v2:type_name -> xatu.eth.v1.EventVoluntaryExitV2 - 112, // 98: xatu.DecoratedEvent.eth_v1_events_contribution_and_proof_v2:type_name -> xatu.eth.v1.EventContributionAndProofV2 - 113, // 99: xatu.DecoratedEvent.eth_v2_beacon_block_v2:type_name -> xatu.eth.v2.EventBlockV2 - 96, // 100: xatu.DecoratedEvent.eth_v1_fork_choice_v2:type_name -> xatu.eth.v1.ForkChoiceV2 - 13, // 101: xatu.DecoratedEvent.eth_v1_fork_choice_reorg_v2:type_name -> xatu.DebugForkChoiceReorgV2 - 114, // 102: xatu.DecoratedEvent.eth_v2_beacon_block_attester_slashing:type_name -> xatu.eth.v1.AttesterSlashingV2 - 115, // 103: xatu.DecoratedEvent.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.eth.v1.ProposerSlashingV2 - 116, // 104: xatu.DecoratedEvent.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.eth.v1.SignedVoluntaryExitV2 - 117, // 105: xatu.DecoratedEvent.eth_v2_beacon_block_deposit:type_name -> xatu.eth.v1.DepositV2 - 118, // 106: xatu.DecoratedEvent.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.eth.v2.SignedBLSToExecutionChangeV2 - 119, // 107: xatu.DecoratedEvent.eth_v2_beacon_block_execution_transaction:type_name -> xatu.eth.v1.Transaction - 120, // 108: xatu.DecoratedEvent.eth_v2_beacon_block_withdrawal:type_name -> xatu.eth.v1.WithdrawalV2 - 121, // 109: xatu.DecoratedEvent.eth_v1_events_blob_sidecar:type_name -> xatu.eth.v1.EventBlobSidecar - 122, // 110: xatu.DecoratedEvent.blockprint_block_classification:type_name -> xatu.blockprint.BlockClassification - 123, // 111: xatu.DecoratedEvent.eth_v1_beacon_block_blob_sidecar:type_name -> xatu.eth.v1.BlobSidecar - 107, // 112: xatu.DecoratedEvent.beacon_p2p_attestation:type_name -> xatu.eth.v1.AttestationV2 - 124, // 113: xatu.DecoratedEvent.eth_v1_proposer_duty:type_name -> xatu.eth.v1.ProposerDuty - 125, // 114: xatu.DecoratedEvent.eth_v2_beacon_block_elaborated_attestation:type_name -> xatu.eth.v1.ElaboratedAttestation - 126, // 115: xatu.DecoratedEvent.libp2p_trace_add_peer:type_name -> xatu.libp2p.AddPeer - 127, // 116: xatu.DecoratedEvent.libp2p_trace_remove_peer:type_name -> xatu.libp2p.RemovePeer - 128, // 117: xatu.DecoratedEvent.libp2p_trace_recv_rpc:type_name -> xatu.libp2p.RecvRPC - 129, // 118: xatu.DecoratedEvent.libp2p_trace_send_rpc:type_name -> xatu.libp2p.SendRPC - 130, // 119: xatu.DecoratedEvent.libp2p_trace_join:type_name -> xatu.libp2p.Join - 131, // 120: xatu.DecoratedEvent.libp2p_trace_connected:type_name -> xatu.libp2p.Connected - 132, // 121: xatu.DecoratedEvent.libp2p_trace_disconnected:type_name -> xatu.libp2p.Disconnected - 133, // 122: xatu.DecoratedEvent.libp2p_trace_handle_metadata:type_name -> xatu.libp2p.HandleMetadata - 134, // 123: xatu.DecoratedEvent.libp2p_trace_handle_status:type_name -> xatu.libp2p.HandleStatus - 135, // 124: xatu.DecoratedEvent.libp2p_trace_gossipsub_beacon_block:type_name -> xatu.libp2p.gossipsub.eth.BeaconBlock - 98, // 125: xatu.DecoratedEvent.libp2p_trace_gossipsub_beacon_attestation:type_name -> xatu.eth.v1.Attestation - 136, // 126: xatu.DecoratedEvent.libp2p_trace_gossipsub_blob_sidecar:type_name -> xatu.libp2p.gossipsub.eth.BlobSidecar - 82, // 127: xatu.ClientMeta.Ethereum.network:type_name -> xatu.ClientMeta.Ethereum.Network - 83, // 128: xatu.ClientMeta.Ethereum.execution:type_name -> xatu.ClientMeta.Ethereum.Execution - 84, // 129: xatu.ClientMeta.Ethereum.consensus:type_name -> xatu.ClientMeta.Ethereum.Consensus - 3, // 130: xatu.ClientMeta.AdditionalEthV1AttestationSourceData.epoch:type_name -> xatu.Epoch - 4, // 131: xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data.epoch:type_name -> xatu.EpochV2 - 3, // 132: xatu.ClientMeta.AdditionalEthV1AttestationTargetData.epoch:type_name -> xatu.Epoch - 4, // 133: xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data.epoch:type_name -> xatu.EpochV2 - 22, // 134: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceData - 24, // 135: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetData - 5, // 136: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.slot:type_name -> xatu.Slot - 3, // 137: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.epoch:type_name -> xatu.Epoch - 8, // 138: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.propagation:type_name -> xatu.Propagation - 10, // 139: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.attesting_validator:type_name -> xatu.AttestingValidator - 23, // 140: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data - 25, // 141: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data - 6, // 142: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.slot:type_name -> xatu.SlotV2 - 4, // 143: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.epoch:type_name -> xatu.EpochV2 - 9, // 144: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.propagation:type_name -> xatu.PropagationV2 - 11, // 145: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.attesting_validator:type_name -> xatu.AttestingValidatorV2 - 3, // 146: xatu.ClientMeta.AdditionalEthV1EventsHeadData.epoch:type_name -> xatu.Epoch - 5, // 147: xatu.ClientMeta.AdditionalEthV1EventsHeadData.slot:type_name -> xatu.Slot - 8, // 148: xatu.ClientMeta.AdditionalEthV1EventsHeadData.propagation:type_name -> xatu.Propagation - 4, // 149: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data.epoch:type_name -> xatu.EpochV2 - 6, // 150: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data.slot:type_name -> xatu.SlotV2 - 9, // 151: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data.propagation:type_name -> xatu.PropagationV2 - 3, // 152: xatu.ClientMeta.AdditionalEthV1EventsBlockData.epoch:type_name -> xatu.Epoch - 5, // 153: xatu.ClientMeta.AdditionalEthV1EventsBlockData.slot:type_name -> xatu.Slot - 8, // 154: xatu.ClientMeta.AdditionalEthV1EventsBlockData.propagation:type_name -> xatu.Propagation - 4, // 155: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data.epoch:type_name -> xatu.EpochV2 - 6, // 156: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data.slot:type_name -> xatu.SlotV2 - 9, // 157: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data.propagation:type_name -> xatu.PropagationV2 - 3, // 158: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitData.epoch:type_name -> xatu.Epoch - 4, // 159: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitV2Data.epoch:type_name -> xatu.EpochV2 - 3, // 160: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointData.epoch:type_name -> xatu.Epoch - 4, // 161: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointV2Data.epoch:type_name -> xatu.EpochV2 - 3, // 162: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData.epoch:type_name -> xatu.Epoch - 5, // 163: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData.slot:type_name -> xatu.Slot - 8, // 164: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData.propagation:type_name -> xatu.Propagation - 4, // 165: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data.epoch:type_name -> xatu.EpochV2 - 6, // 166: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data.slot:type_name -> xatu.SlotV2 - 9, // 167: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data.propagation:type_name -> xatu.PropagationV2 - 3, // 168: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData.epoch:type_name -> xatu.Epoch - 5, // 169: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData.slot:type_name -> xatu.Slot - 8, // 170: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData.propagation:type_name -> xatu.Propagation - 4, // 171: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data.epoch:type_name -> xatu.EpochV2 - 6, // 172: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data.slot:type_name -> xatu.SlotV2 - 9, // 173: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data.propagation:type_name -> xatu.PropagationV2 - 38, // 174: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofData.contribution:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData - 39, // 175: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofV2Data.contribution:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data - 3, // 176: xatu.ClientMeta.ForkChoiceSnapshot.request_epoch:type_name -> xatu.Epoch - 5, // 177: xatu.ClientMeta.ForkChoiceSnapshot.request_slot:type_name -> xatu.Slot - 92, // 178: xatu.ClientMeta.ForkChoiceSnapshot.timestamp:type_name -> google.protobuf.Timestamp - 4, // 179: xatu.ClientMeta.ForkChoiceSnapshotV2.request_epoch:type_name -> xatu.EpochV2 - 6, // 180: xatu.ClientMeta.ForkChoiceSnapshotV2.request_slot:type_name -> xatu.SlotV2 - 93, // 181: xatu.ClientMeta.ForkChoiceSnapshotV2.requested_at_slot_start_diff_ms:type_name -> google.protobuf.UInt64Value - 93, // 182: xatu.ClientMeta.ForkChoiceSnapshotV2.request_duration_ms:type_name -> google.protobuf.UInt64Value - 92, // 183: xatu.ClientMeta.ForkChoiceSnapshotV2.timestamp:type_name -> google.protobuf.Timestamp - 42, // 184: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceData.Snapshot:type_name -> xatu.ClientMeta.ForkChoiceSnapshot - 43, // 185: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceV2Data.Snapshot:type_name -> xatu.ClientMeta.ForkChoiceSnapshotV2 - 42, // 186: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData.before:type_name -> xatu.ClientMeta.ForkChoiceSnapshot - 42, // 187: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData.after:type_name -> xatu.ClientMeta.ForkChoiceSnapshot - 43, // 188: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data.before:type_name -> xatu.ClientMeta.ForkChoiceSnapshotV2 - 43, // 189: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data.after:type_name -> xatu.ClientMeta.ForkChoiceSnapshotV2 - 4, // 190: xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData.epoch:type_name -> xatu.EpochV2 - 6, // 191: xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData.slot:type_name -> xatu.SlotV2 - 93, // 192: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.nonce:type_name -> google.protobuf.UInt64Value - 93, // 193: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.gas:type_name -> google.protobuf.UInt64Value - 137, // 194: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.type:type_name -> google.protobuf.UInt32Value - 93, // 195: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.blob_gas:type_name -> google.protobuf.UInt64Value - 3, // 196: xatu.ClientMeta.AdditionalEthV2BeaconBlockData.epoch:type_name -> xatu.Epoch - 5, // 197: xatu.ClientMeta.AdditionalEthV2BeaconBlockData.slot:type_name -> xatu.Slot - 4, // 198: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.epoch:type_name -> xatu.EpochV2 - 6, // 199: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.slot:type_name -> xatu.SlotV2 - 93, // 200: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.transactions_count:type_name -> google.protobuf.UInt64Value - 93, // 201: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.transactions_total_bytes:type_name -> google.protobuf.UInt64Value - 93, // 202: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.transactions_total_bytes_compressed:type_name -> google.protobuf.UInt64Value - 93, // 203: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.total_bytes:type_name -> google.protobuf.UInt64Value - 93, // 204: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.total_bytes_compressed:type_name -> google.protobuf.UInt64Value - 14, // 205: xatu.ClientMeta.AdditionalEthV2BeaconBlockAttesterSlashingData.block:type_name -> xatu.BlockIdentifier - 14, // 206: xatu.ClientMeta.AdditionalEthV2BeaconBlockProposerSlashingData.block:type_name -> xatu.BlockIdentifier - 14, // 207: xatu.ClientMeta.AdditionalEthV2BeaconBlockVoluntaryExitData.block:type_name -> xatu.BlockIdentifier - 14, // 208: xatu.ClientMeta.AdditionalEthV2BeaconBlockDepositData.block:type_name -> xatu.BlockIdentifier - 14, // 209: xatu.ClientMeta.AdditionalEthV2BeaconBlockBLSToExecutionChangeData.block:type_name -> xatu.BlockIdentifier - 14, // 210: xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData.block:type_name -> xatu.BlockIdentifier - 93, // 211: xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData.position_in_block:type_name -> google.protobuf.UInt64Value - 14, // 212: xatu.ClientMeta.AdditionalEthV2BeaconBlockWithdrawalData.block:type_name -> xatu.BlockIdentifier - 4, // 213: xatu.ClientMeta.AdditionalBlockprintBlockClassificationData.epoch:type_name -> xatu.EpochV2 - 6, // 214: xatu.ClientMeta.AdditionalBlockprintBlockClassificationData.slot:type_name -> xatu.SlotV2 - 93, // 215: xatu.ClientMeta.AttestationDataSnapshot.requested_at_slot_start_diff_ms:type_name -> google.protobuf.UInt64Value - 93, // 216: xatu.ClientMeta.AttestationDataSnapshot.request_duration_ms:type_name -> google.protobuf.UInt64Value - 92, // 217: xatu.ClientMeta.AttestationDataSnapshot.timestamp:type_name -> google.protobuf.Timestamp - 23, // 218: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data - 25, // 219: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data - 4, // 220: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.epoch:type_name -> xatu.EpochV2 - 6, // 221: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.slot:type_name -> xatu.SlotV2 - 61, // 222: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.Snapshot:type_name -> xatu.ClientMeta.AttestationDataSnapshot - 4, // 223: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData.epoch:type_name -> xatu.EpochV2 - 6, // 224: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData.slot:type_name -> xatu.SlotV2 - 9, // 225: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData.propagation:type_name -> xatu.PropagationV2 - 4, // 226: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.epoch:type_name -> xatu.EpochV2 - 6, // 227: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.slot:type_name -> xatu.SlotV2 - 93, // 228: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.data_size:type_name -> google.protobuf.UInt64Value - 93, // 229: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.data_empty_size:type_name -> google.protobuf.UInt64Value - 23, // 230: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data - 25, // 231: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data - 6, // 232: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.slot:type_name -> xatu.SlotV2 - 4, // 233: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.epoch:type_name -> xatu.EpochV2 - 9, // 234: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.propagation:type_name -> xatu.PropagationV2 - 11, // 235: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.attesting_validator:type_name -> xatu.AttestingValidatorV2 - 138, // 236: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.peer:type_name -> xatu.libp2p.Peer - 137, // 237: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.subnet:type_name -> google.protobuf.UInt32Value - 139, // 238: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.validated:type_name -> google.protobuf.BoolValue - 4, // 239: xatu.ClientMeta.AdditionalEthV1ProposerDutyData.epoch:type_name -> xatu.EpochV2 - 6, // 240: xatu.ClientMeta.AdditionalEthV1ProposerDutyData.slot:type_name -> xatu.SlotV2 - 14, // 241: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.block:type_name -> xatu.BlockIdentifier - 93, // 242: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.position_in_block:type_name -> google.protobuf.UInt64Value - 4, // 243: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.epoch:type_name -> xatu.EpochV2 - 6, // 244: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.slot:type_name -> xatu.SlotV2 - 23, // 245: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data - 25, // 246: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data - 140, // 247: xatu.ClientMeta.AdditionalLibP2PTraceAddPeerData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 248: xatu.ClientMeta.AdditionalLibP2PTraceRemovePeerData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 249: xatu.ClientMeta.AdditionalLibP2PTraceRecvRPCData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 250: xatu.ClientMeta.AdditionalLibP2PTraceSendRPCData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 251: xatu.ClientMeta.AdditionalLibP2PTraceJoinData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 252: xatu.ClientMeta.AdditionalLibP2PTraceConnectedData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 253: xatu.ClientMeta.AdditionalLibP2PTraceDisconnectedData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 254: xatu.ClientMeta.AdditionalLibP2PTraceHandleMetadataData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 140, // 255: xatu.ClientMeta.AdditionalLibP2PTraceHandleStatusData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 4, // 256: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.epoch:type_name -> xatu.EpochV2 - 6, // 257: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.slot:type_name -> xatu.SlotV2 - 4, // 258: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.wallclock_epoch:type_name -> xatu.EpochV2 - 6, // 259: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.wallclock_slot:type_name -> xatu.SlotV2 - 9, // 260: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.propagation:type_name -> xatu.PropagationV2 - 140, // 261: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 141, // 262: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.topic:type_name -> google.protobuf.StringValue - 137, // 263: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.message_size:type_name -> google.protobuf.UInt32Value - 141, // 264: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.message_id:type_name -> google.protobuf.StringValue - 4, // 265: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData.epoch:type_name -> xatu.EpochV2 - 4, // 266: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData.epoch:type_name -> xatu.EpochV2 - 78, // 267: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.source:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData - 79, // 268: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.target:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData - 6, // 269: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.slot:type_name -> xatu.SlotV2 - 4, // 270: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.epoch:type_name -> xatu.EpochV2 - 9, // 271: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.propagation:type_name -> xatu.PropagationV2 - 11, // 272: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.attesting_validator:type_name -> xatu.AttestingValidatorV2 - 4, // 273: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.wallclock_epoch:type_name -> xatu.EpochV2 - 6, // 274: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.wallclock_slot:type_name -> xatu.SlotV2 - 140, // 275: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 141, // 276: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.topic:type_name -> google.protobuf.StringValue - 137, // 277: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.message_size:type_name -> google.protobuf.UInt32Value - 141, // 278: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.message_id:type_name -> google.protobuf.StringValue - 4, // 279: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.epoch:type_name -> xatu.EpochV2 - 6, // 280: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.slot:type_name -> xatu.SlotV2 - 4, // 281: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.wallclock_epoch:type_name -> xatu.EpochV2 - 6, // 282: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.wallclock_slot:type_name -> xatu.SlotV2 - 9, // 283: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.propagation:type_name -> xatu.PropagationV2 - 140, // 284: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.metadata:type_name -> xatu.libp2p.TraceEventMetadata - 141, // 285: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.topic:type_name -> google.protobuf.StringValue - 137, // 286: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.message_size:type_name -> google.protobuf.UInt32Value - 141, // 287: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.message_id:type_name -> google.protobuf.StringValue - 7, // 288: xatu.ClientMeta.Ethereum.Execution.fork_id:type_name -> xatu.ForkID - 92, // 289: xatu.ServerMeta.Event.received_date_time:type_name -> google.protobuf.Timestamp - 86, // 290: xatu.ServerMeta.Client.geo:type_name -> xatu.ServerMeta.Geo - 86, // 291: xatu.ServerMeta.Peer.geo:type_name -> xatu.ServerMeta.Geo - 88, // 292: xatu.ServerMeta.AdditionalBeaconP2PAttestationData.peer:type_name -> xatu.ServerMeta.Peer - 88, // 293: xatu.ServerMeta.AdditionalLibp2PTraceConnectedData.peer:type_name -> xatu.ServerMeta.Peer - 88, // 294: xatu.ServerMeta.AdditionalLibp2PTraceDisconnectedData.peer:type_name -> xatu.ServerMeta.Peer - 1, // 295: xatu.EventIngester.CreateEvents:input_type -> xatu.CreateEventsRequest - 2, // 296: xatu.EventIngester.CreateEvents:output_type -> xatu.CreateEventsResponse - 296, // [296:297] is the sub-list for method output_type - 295, // [295:296] is the sub-list for method input_type - 295, // [295:295] is the sub-list for extension type_name - 295, // [295:295] is the sub-list for extension extendee - 0, // [0:295] is the sub-list for field type_name + 20, // 0: xatu.CreateEventsRequest.events:type_name -> xatu.DecoratedEvent + 94, // 1: xatu.Epoch.start_date_time:type_name -> google.protobuf.Timestamp + 95, // 2: xatu.EpochV2.number:type_name -> google.protobuf.UInt64Value + 94, // 3: xatu.EpochV2.start_date_time:type_name -> google.protobuf.Timestamp + 94, // 4: xatu.Slot.start_date_time:type_name -> google.protobuf.Timestamp + 95, // 5: xatu.SlotV2.number:type_name -> google.protobuf.UInt64Value + 94, // 6: xatu.SlotV2.start_date_time:type_name -> google.protobuf.Timestamp + 95, // 7: xatu.PropagationV2.slot_start_diff:type_name -> google.protobuf.UInt64Value + 95, // 8: xatu.AttestingValidatorV2.committee_index:type_name -> google.protobuf.UInt64Value + 95, // 9: xatu.AttestingValidatorV2.index:type_name -> google.protobuf.UInt64Value + 96, // 10: xatu.DebugForkChoiceReorg.before:type_name -> xatu.eth.v1.ForkChoice + 96, // 11: xatu.DebugForkChoiceReorg.after:type_name -> xatu.eth.v1.ForkChoice + 97, // 12: xatu.DebugForkChoiceReorg.event:type_name -> xatu.eth.v1.EventChainReorg + 98, // 13: xatu.DebugForkChoiceReorgV2.before:type_name -> xatu.eth.v1.ForkChoiceV2 + 98, // 14: xatu.DebugForkChoiceReorgV2.after:type_name -> xatu.eth.v1.ForkChoiceV2 + 99, // 15: xatu.DebugForkChoiceReorgV2.event:type_name -> xatu.eth.v1.EventChainReorgV2 + 100, // 16: xatu.Validators.validators:type_name -> xatu.eth.v1.Validator + 4, // 17: xatu.BlockIdentifier.epoch:type_name -> xatu.EpochV2 + 6, // 18: xatu.BlockIdentifier.slot:type_name -> xatu.SlotV2 + 21, // 19: xatu.ClientMeta.ethereum:type_name -> xatu.ClientMeta.Ethereum + 22, // 20: xatu.ClientMeta.labels:type_name -> xatu.ClientMeta.LabelsEntry + 27, // 21: xatu.ClientMeta.eth_v1_events_attestation:type_name -> xatu.ClientMeta.AdditionalEthV1EventsAttestationData + 29, // 22: xatu.ClientMeta.eth_v1_events_head:type_name -> xatu.ClientMeta.AdditionalEthV1EventsHeadData + 31, // 23: xatu.ClientMeta.eth_v1_events_block:type_name -> xatu.ClientMeta.AdditionalEthV1EventsBlockData + 33, // 24: xatu.ClientMeta.eth_v1_events_voluntary_exit:type_name -> xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitData + 35, // 25: xatu.ClientMeta.eth_v1_events_finalized_checkpoint:type_name -> xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointData + 37, // 26: xatu.ClientMeta.eth_v1_events_chain_reorg:type_name -> xatu.ClientMeta.AdditionalEthV1EventsChainReorgData + 41, // 27: xatu.ClientMeta.eth_v1_events_contribution_and_proof:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofData + 50, // 28: xatu.ClientMeta.mempool_transaction:type_name -> xatu.ClientMeta.AdditionalMempoolTransactionData + 52, // 29: xatu.ClientMeta.eth_v2_beacon_block:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockData + 45, // 30: xatu.ClientMeta.eth_v1_debug_fork_choice:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceData + 47, // 31: xatu.ClientMeta.eth_v1_debug_fork_choice_reorg:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData + 49, // 32: xatu.ClientMeta.eth_v1_beacon_committee:type_name -> xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData + 63, // 33: xatu.ClientMeta.eth_v1_validator_attestation_data:type_name -> xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData + 28, // 34: xatu.ClientMeta.eth_v1_events_attestation_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data + 30, // 35: xatu.ClientMeta.eth_v1_events_head_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data + 32, // 36: xatu.ClientMeta.eth_v1_events_block_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data + 34, // 37: xatu.ClientMeta.eth_v1_events_voluntary_exit_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitV2Data + 36, // 38: xatu.ClientMeta.eth_v1_events_finalized_checkpoint_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointV2Data + 38, // 39: xatu.ClientMeta.eth_v1_events_chain_reorg_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data + 42, // 40: xatu.ClientMeta.eth_v1_events_contribution_and_proof_v2:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofV2Data + 51, // 41: xatu.ClientMeta.mempool_transaction_v2:type_name -> xatu.ClientMeta.AdditionalMempoolTransactionV2Data + 53, // 42: xatu.ClientMeta.eth_v2_beacon_block_v2:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data + 46, // 43: xatu.ClientMeta.eth_v1_debug_fork_choice_v2:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceV2Data + 48, // 44: xatu.ClientMeta.eth_v1_debug_fork_choice_reorg_v2:type_name -> xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data + 54, // 45: xatu.ClientMeta.eth_v2_beacon_block_attester_slashing:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockAttesterSlashingData + 55, // 46: xatu.ClientMeta.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockProposerSlashingData + 56, // 47: xatu.ClientMeta.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockVoluntaryExitData + 57, // 48: xatu.ClientMeta.eth_v2_beacon_block_deposit:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockDepositData + 58, // 49: xatu.ClientMeta.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockBLSToExecutionChangeData + 59, // 50: xatu.ClientMeta.eth_v2_beacon_block_execution_transaction:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData + 60, // 51: xatu.ClientMeta.eth_v2_beacon_block_withdrawal:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockWithdrawalData + 64, // 52: xatu.ClientMeta.eth_v1_events_blob_sidecar:type_name -> xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData + 61, // 53: xatu.ClientMeta.blockprint_block_classification:type_name -> xatu.ClientMeta.AdditionalBlockprintBlockClassificationData + 65, // 54: xatu.ClientMeta.eth_v1_beacon_blob_sidecar:type_name -> xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData + 66, // 55: xatu.ClientMeta.beacon_p2p_attestation:type_name -> xatu.ClientMeta.AdditionalBeaconP2PAttestationData + 67, // 56: xatu.ClientMeta.eth_v1_proposer_duty:type_name -> xatu.ClientMeta.AdditionalEthV1ProposerDutyData + 68, // 57: xatu.ClientMeta.eth_v2_beacon_block_elaborated_attestation:type_name -> xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData + 69, // 58: xatu.ClientMeta.libp2p_trace_add_peer:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceAddPeerData + 70, // 59: xatu.ClientMeta.libp2p_trace_remove_peer:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceRemovePeerData + 71, // 60: xatu.ClientMeta.libp2p_trace_recv_rpc:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceRecvRPCData + 72, // 61: xatu.ClientMeta.libp2p_trace_send_rpc:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceSendRPCData + 73, // 62: xatu.ClientMeta.libp2p_trace_join:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceJoinData + 74, // 63: xatu.ClientMeta.libp2p_trace_connected:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceConnectedData + 75, // 64: xatu.ClientMeta.libp2p_trace_disconnected:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceDisconnectedData + 76, // 65: xatu.ClientMeta.libp2p_trace_handle_metadata:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceHandleMetadataData + 77, // 66: xatu.ClientMeta.libp2p_trace_handle_status:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceHandleStatusData + 78, // 67: xatu.ClientMeta.libp2p_trace_gossipsub_beacon_block:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData + 81, // 68: xatu.ClientMeta.libp2p_trace_gossipsub_beacon_attestation:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData + 82, // 69: xatu.ClientMeta.libp2p_trace_gossipsub_blob_sidecar:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData + 83, // 70: xatu.ClientMeta.eth_v1_validators:type_name -> xatu.ClientMeta.AdditionalEthV1ValidatorsData + 87, // 71: xatu.ServerMeta.event:type_name -> xatu.ServerMeta.Event + 89, // 72: xatu.ServerMeta.client:type_name -> xatu.ServerMeta.Client + 91, // 73: xatu.ServerMeta.BEACON_P2P_ATTESTATION:type_name -> xatu.ServerMeta.AdditionalBeaconP2PAttestationData + 92, // 74: xatu.ServerMeta.LIBP2P_TRACE_CONNECTED:type_name -> xatu.ServerMeta.AdditionalLibp2PTraceConnectedData + 93, // 75: xatu.ServerMeta.LIBP2P_TRACE_DISCONNECTED:type_name -> xatu.ServerMeta.AdditionalLibp2PTraceDisconnectedData + 16, // 76: xatu.Meta.client:type_name -> xatu.ClientMeta + 17, // 77: xatu.Meta.server:type_name -> xatu.ServerMeta + 0, // 78: xatu.Event.name:type_name -> xatu.Event.Name + 94, // 79: xatu.Event.date_time:type_name -> google.protobuf.Timestamp + 19, // 80: xatu.DecoratedEvent.event:type_name -> xatu.Event + 18, // 81: xatu.DecoratedEvent.meta:type_name -> xatu.Meta + 101, // 82: xatu.DecoratedEvent.eth_v1_events_attestation:type_name -> xatu.eth.v1.Attestation + 102, // 83: xatu.DecoratedEvent.eth_v1_events_block:type_name -> xatu.eth.v1.EventBlock + 97, // 84: xatu.DecoratedEvent.eth_v1_events_chain_reorg:type_name -> xatu.eth.v1.EventChainReorg + 103, // 85: xatu.DecoratedEvent.eth_v1_events_finalized_checkpoint:type_name -> xatu.eth.v1.EventFinalizedCheckpoint + 104, // 86: xatu.DecoratedEvent.eth_v1_events_head:type_name -> xatu.eth.v1.EventHead + 105, // 87: xatu.DecoratedEvent.eth_v1_events_voluntary_exit:type_name -> xatu.eth.v1.EventVoluntaryExit + 106, // 88: xatu.DecoratedEvent.eth_v1_events_contribution_and_proof:type_name -> xatu.eth.v1.EventContributionAndProof + 107, // 89: xatu.DecoratedEvent.eth_v2_beacon_block:type_name -> xatu.eth.v2.EventBlock + 96, // 90: xatu.DecoratedEvent.eth_v1_fork_choice:type_name -> xatu.eth.v1.ForkChoice + 12, // 91: xatu.DecoratedEvent.eth_v1_fork_choice_reorg:type_name -> xatu.DebugForkChoiceReorg + 108, // 92: xatu.DecoratedEvent.eth_v1_beacon_committee:type_name -> xatu.eth.v1.Committee + 109, // 93: xatu.DecoratedEvent.eth_v1_validator_attestation_data:type_name -> xatu.eth.v1.AttestationDataV2 + 110, // 94: xatu.DecoratedEvent.eth_v1_events_attestation_v2:type_name -> xatu.eth.v1.AttestationV2 + 111, // 95: xatu.DecoratedEvent.eth_v1_events_block_v2:type_name -> xatu.eth.v1.EventBlockV2 + 99, // 96: xatu.DecoratedEvent.eth_v1_events_chain_reorg_v2:type_name -> xatu.eth.v1.EventChainReorgV2 + 112, // 97: xatu.DecoratedEvent.eth_v1_events_finalized_checkpoint_v2:type_name -> xatu.eth.v1.EventFinalizedCheckpointV2 + 113, // 98: xatu.DecoratedEvent.eth_v1_events_head_v2:type_name -> xatu.eth.v1.EventHeadV2 + 114, // 99: xatu.DecoratedEvent.eth_v1_events_voluntary_exit_v2:type_name -> xatu.eth.v1.EventVoluntaryExitV2 + 115, // 100: xatu.DecoratedEvent.eth_v1_events_contribution_and_proof_v2:type_name -> xatu.eth.v1.EventContributionAndProofV2 + 116, // 101: xatu.DecoratedEvent.eth_v2_beacon_block_v2:type_name -> xatu.eth.v2.EventBlockV2 + 98, // 102: xatu.DecoratedEvent.eth_v1_fork_choice_v2:type_name -> xatu.eth.v1.ForkChoiceV2 + 13, // 103: xatu.DecoratedEvent.eth_v1_fork_choice_reorg_v2:type_name -> xatu.DebugForkChoiceReorgV2 + 117, // 104: xatu.DecoratedEvent.eth_v2_beacon_block_attester_slashing:type_name -> xatu.eth.v1.AttesterSlashingV2 + 118, // 105: xatu.DecoratedEvent.eth_v2_beacon_block_proposer_slashing:type_name -> xatu.eth.v1.ProposerSlashingV2 + 119, // 106: xatu.DecoratedEvent.eth_v2_beacon_block_voluntary_exit:type_name -> xatu.eth.v1.SignedVoluntaryExitV2 + 120, // 107: xatu.DecoratedEvent.eth_v2_beacon_block_deposit:type_name -> xatu.eth.v1.DepositV2 + 121, // 108: xatu.DecoratedEvent.eth_v2_beacon_block_bls_to_execution_change:type_name -> xatu.eth.v2.SignedBLSToExecutionChangeV2 + 122, // 109: xatu.DecoratedEvent.eth_v2_beacon_block_execution_transaction:type_name -> xatu.eth.v1.Transaction + 123, // 110: xatu.DecoratedEvent.eth_v2_beacon_block_withdrawal:type_name -> xatu.eth.v1.WithdrawalV2 + 124, // 111: xatu.DecoratedEvent.eth_v1_events_blob_sidecar:type_name -> xatu.eth.v1.EventBlobSidecar + 125, // 112: xatu.DecoratedEvent.blockprint_block_classification:type_name -> xatu.blockprint.BlockClassification + 126, // 113: xatu.DecoratedEvent.eth_v1_beacon_block_blob_sidecar:type_name -> xatu.eth.v1.BlobSidecar + 110, // 114: xatu.DecoratedEvent.beacon_p2p_attestation:type_name -> xatu.eth.v1.AttestationV2 + 127, // 115: xatu.DecoratedEvent.eth_v1_proposer_duty:type_name -> xatu.eth.v1.ProposerDuty + 128, // 116: xatu.DecoratedEvent.eth_v2_beacon_block_elaborated_attestation:type_name -> xatu.eth.v1.ElaboratedAttestation + 129, // 117: xatu.DecoratedEvent.libp2p_trace_add_peer:type_name -> xatu.libp2p.AddPeer + 130, // 118: xatu.DecoratedEvent.libp2p_trace_remove_peer:type_name -> xatu.libp2p.RemovePeer + 131, // 119: xatu.DecoratedEvent.libp2p_trace_recv_rpc:type_name -> xatu.libp2p.RecvRPC + 132, // 120: xatu.DecoratedEvent.libp2p_trace_send_rpc:type_name -> xatu.libp2p.SendRPC + 133, // 121: xatu.DecoratedEvent.libp2p_trace_join:type_name -> xatu.libp2p.Join + 134, // 122: xatu.DecoratedEvent.libp2p_trace_connected:type_name -> xatu.libp2p.Connected + 135, // 123: xatu.DecoratedEvent.libp2p_trace_disconnected:type_name -> xatu.libp2p.Disconnected + 136, // 124: xatu.DecoratedEvent.libp2p_trace_handle_metadata:type_name -> xatu.libp2p.HandleMetadata + 137, // 125: xatu.DecoratedEvent.libp2p_trace_handle_status:type_name -> xatu.libp2p.HandleStatus + 138, // 126: xatu.DecoratedEvent.libp2p_trace_gossipsub_beacon_block:type_name -> xatu.libp2p.gossipsub.eth.BeaconBlock + 101, // 127: xatu.DecoratedEvent.libp2p_trace_gossipsub_beacon_attestation:type_name -> xatu.eth.v1.Attestation + 139, // 128: xatu.DecoratedEvent.libp2p_trace_gossipsub_blob_sidecar:type_name -> xatu.libp2p.gossipsub.eth.BlobSidecar + 14, // 129: xatu.DecoratedEvent.eth_v1_validators:type_name -> xatu.Validators + 84, // 130: xatu.ClientMeta.Ethereum.network:type_name -> xatu.ClientMeta.Ethereum.Network + 85, // 131: xatu.ClientMeta.Ethereum.execution:type_name -> xatu.ClientMeta.Ethereum.Execution + 86, // 132: xatu.ClientMeta.Ethereum.consensus:type_name -> xatu.ClientMeta.Ethereum.Consensus + 3, // 133: xatu.ClientMeta.AdditionalEthV1AttestationSourceData.epoch:type_name -> xatu.Epoch + 4, // 134: xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data.epoch:type_name -> xatu.EpochV2 + 3, // 135: xatu.ClientMeta.AdditionalEthV1AttestationTargetData.epoch:type_name -> xatu.Epoch + 4, // 136: xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data.epoch:type_name -> xatu.EpochV2 + 23, // 137: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceData + 25, // 138: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetData + 5, // 139: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.slot:type_name -> xatu.Slot + 3, // 140: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.epoch:type_name -> xatu.Epoch + 8, // 141: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.propagation:type_name -> xatu.Propagation + 10, // 142: xatu.ClientMeta.AdditionalEthV1EventsAttestationData.attesting_validator:type_name -> xatu.AttestingValidator + 24, // 143: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data + 26, // 144: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data + 6, // 145: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.slot:type_name -> xatu.SlotV2 + 4, // 146: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.epoch:type_name -> xatu.EpochV2 + 9, // 147: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.propagation:type_name -> xatu.PropagationV2 + 11, // 148: xatu.ClientMeta.AdditionalEthV1EventsAttestationV2Data.attesting_validator:type_name -> xatu.AttestingValidatorV2 + 3, // 149: xatu.ClientMeta.AdditionalEthV1EventsHeadData.epoch:type_name -> xatu.Epoch + 5, // 150: xatu.ClientMeta.AdditionalEthV1EventsHeadData.slot:type_name -> xatu.Slot + 8, // 151: xatu.ClientMeta.AdditionalEthV1EventsHeadData.propagation:type_name -> xatu.Propagation + 4, // 152: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data.epoch:type_name -> xatu.EpochV2 + 6, // 153: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data.slot:type_name -> xatu.SlotV2 + 9, // 154: xatu.ClientMeta.AdditionalEthV1EventsHeadV2Data.propagation:type_name -> xatu.PropagationV2 + 3, // 155: xatu.ClientMeta.AdditionalEthV1EventsBlockData.epoch:type_name -> xatu.Epoch + 5, // 156: xatu.ClientMeta.AdditionalEthV1EventsBlockData.slot:type_name -> xatu.Slot + 8, // 157: xatu.ClientMeta.AdditionalEthV1EventsBlockData.propagation:type_name -> xatu.Propagation + 4, // 158: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data.epoch:type_name -> xatu.EpochV2 + 6, // 159: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data.slot:type_name -> xatu.SlotV2 + 9, // 160: xatu.ClientMeta.AdditionalEthV1EventsBlockV2Data.propagation:type_name -> xatu.PropagationV2 + 3, // 161: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitData.epoch:type_name -> xatu.Epoch + 4, // 162: xatu.ClientMeta.AdditionalEthV1EventsVoluntaryExitV2Data.epoch:type_name -> xatu.EpochV2 + 3, // 163: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointData.epoch:type_name -> xatu.Epoch + 4, // 164: xatu.ClientMeta.AdditionalEthV1EventsFinalizedCheckpointV2Data.epoch:type_name -> xatu.EpochV2 + 3, // 165: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData.epoch:type_name -> xatu.Epoch + 5, // 166: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData.slot:type_name -> xatu.Slot + 8, // 167: xatu.ClientMeta.AdditionalEthV1EventsChainReorgData.propagation:type_name -> xatu.Propagation + 4, // 168: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data.epoch:type_name -> xatu.EpochV2 + 6, // 169: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data.slot:type_name -> xatu.SlotV2 + 9, // 170: xatu.ClientMeta.AdditionalEthV1EventsChainReorgV2Data.propagation:type_name -> xatu.PropagationV2 + 3, // 171: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData.epoch:type_name -> xatu.Epoch + 5, // 172: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData.slot:type_name -> xatu.Slot + 8, // 173: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData.propagation:type_name -> xatu.Propagation + 4, // 174: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data.epoch:type_name -> xatu.EpochV2 + 6, // 175: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data.slot:type_name -> xatu.SlotV2 + 9, // 176: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data.propagation:type_name -> xatu.PropagationV2 + 39, // 177: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofData.contribution:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionData + 40, // 178: xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofV2Data.contribution:type_name -> xatu.ClientMeta.AdditionalEthV1EventsContributionAndProofContributionV2Data + 3, // 179: xatu.ClientMeta.ForkChoiceSnapshot.request_epoch:type_name -> xatu.Epoch + 5, // 180: xatu.ClientMeta.ForkChoiceSnapshot.request_slot:type_name -> xatu.Slot + 94, // 181: xatu.ClientMeta.ForkChoiceSnapshot.timestamp:type_name -> google.protobuf.Timestamp + 4, // 182: xatu.ClientMeta.ForkChoiceSnapshotV2.request_epoch:type_name -> xatu.EpochV2 + 6, // 183: xatu.ClientMeta.ForkChoiceSnapshotV2.request_slot:type_name -> xatu.SlotV2 + 95, // 184: xatu.ClientMeta.ForkChoiceSnapshotV2.requested_at_slot_start_diff_ms:type_name -> google.protobuf.UInt64Value + 95, // 185: xatu.ClientMeta.ForkChoiceSnapshotV2.request_duration_ms:type_name -> google.protobuf.UInt64Value + 94, // 186: xatu.ClientMeta.ForkChoiceSnapshotV2.timestamp:type_name -> google.protobuf.Timestamp + 43, // 187: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceData.Snapshot:type_name -> xatu.ClientMeta.ForkChoiceSnapshot + 44, // 188: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceV2Data.Snapshot:type_name -> xatu.ClientMeta.ForkChoiceSnapshotV2 + 43, // 189: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData.before:type_name -> xatu.ClientMeta.ForkChoiceSnapshot + 43, // 190: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgData.after:type_name -> xatu.ClientMeta.ForkChoiceSnapshot + 44, // 191: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data.before:type_name -> xatu.ClientMeta.ForkChoiceSnapshotV2 + 44, // 192: xatu.ClientMeta.AdditionalEthV1DebugForkChoiceReOrgV2Data.after:type_name -> xatu.ClientMeta.ForkChoiceSnapshotV2 + 4, // 193: xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData.epoch:type_name -> xatu.EpochV2 + 6, // 194: xatu.ClientMeta.AdditionalEthV1BeaconCommitteeData.slot:type_name -> xatu.SlotV2 + 95, // 195: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.nonce:type_name -> google.protobuf.UInt64Value + 95, // 196: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.gas:type_name -> google.protobuf.UInt64Value + 140, // 197: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.type:type_name -> google.protobuf.UInt32Value + 95, // 198: xatu.ClientMeta.AdditionalMempoolTransactionV2Data.blob_gas:type_name -> google.protobuf.UInt64Value + 3, // 199: xatu.ClientMeta.AdditionalEthV2BeaconBlockData.epoch:type_name -> xatu.Epoch + 5, // 200: xatu.ClientMeta.AdditionalEthV2BeaconBlockData.slot:type_name -> xatu.Slot + 4, // 201: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.epoch:type_name -> xatu.EpochV2 + 6, // 202: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.slot:type_name -> xatu.SlotV2 + 95, // 203: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.transactions_count:type_name -> google.protobuf.UInt64Value + 95, // 204: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.transactions_total_bytes:type_name -> google.protobuf.UInt64Value + 95, // 205: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.transactions_total_bytes_compressed:type_name -> google.protobuf.UInt64Value + 95, // 206: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.total_bytes:type_name -> google.protobuf.UInt64Value + 95, // 207: xatu.ClientMeta.AdditionalEthV2BeaconBlockV2Data.total_bytes_compressed:type_name -> google.protobuf.UInt64Value + 15, // 208: xatu.ClientMeta.AdditionalEthV2BeaconBlockAttesterSlashingData.block:type_name -> xatu.BlockIdentifier + 15, // 209: xatu.ClientMeta.AdditionalEthV2BeaconBlockProposerSlashingData.block:type_name -> xatu.BlockIdentifier + 15, // 210: xatu.ClientMeta.AdditionalEthV2BeaconBlockVoluntaryExitData.block:type_name -> xatu.BlockIdentifier + 15, // 211: xatu.ClientMeta.AdditionalEthV2BeaconBlockDepositData.block:type_name -> xatu.BlockIdentifier + 15, // 212: xatu.ClientMeta.AdditionalEthV2BeaconBlockBLSToExecutionChangeData.block:type_name -> xatu.BlockIdentifier + 15, // 213: xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData.block:type_name -> xatu.BlockIdentifier + 95, // 214: xatu.ClientMeta.AdditionalEthV2BeaconBlockExecutionTransactionData.position_in_block:type_name -> google.protobuf.UInt64Value + 15, // 215: xatu.ClientMeta.AdditionalEthV2BeaconBlockWithdrawalData.block:type_name -> xatu.BlockIdentifier + 4, // 216: xatu.ClientMeta.AdditionalBlockprintBlockClassificationData.epoch:type_name -> xatu.EpochV2 + 6, // 217: xatu.ClientMeta.AdditionalBlockprintBlockClassificationData.slot:type_name -> xatu.SlotV2 + 95, // 218: xatu.ClientMeta.AttestationDataSnapshot.requested_at_slot_start_diff_ms:type_name -> google.protobuf.UInt64Value + 95, // 219: xatu.ClientMeta.AttestationDataSnapshot.request_duration_ms:type_name -> google.protobuf.UInt64Value + 94, // 220: xatu.ClientMeta.AttestationDataSnapshot.timestamp:type_name -> google.protobuf.Timestamp + 24, // 221: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data + 26, // 222: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data + 4, // 223: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.epoch:type_name -> xatu.EpochV2 + 6, // 224: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.slot:type_name -> xatu.SlotV2 + 62, // 225: xatu.ClientMeta.AdditionalEthV1ValidatorAttestationDataData.Snapshot:type_name -> xatu.ClientMeta.AttestationDataSnapshot + 4, // 226: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData.epoch:type_name -> xatu.EpochV2 + 6, // 227: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData.slot:type_name -> xatu.SlotV2 + 9, // 228: xatu.ClientMeta.AdditionalEthV1EventsBlobSidecarData.propagation:type_name -> xatu.PropagationV2 + 4, // 229: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.epoch:type_name -> xatu.EpochV2 + 6, // 230: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.slot:type_name -> xatu.SlotV2 + 95, // 231: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.data_size:type_name -> google.protobuf.UInt64Value + 95, // 232: xatu.ClientMeta.AdditionalEthV1BeaconBlobSidecarData.data_empty_size:type_name -> google.protobuf.UInt64Value + 24, // 233: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data + 26, // 234: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data + 6, // 235: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.slot:type_name -> xatu.SlotV2 + 4, // 236: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.epoch:type_name -> xatu.EpochV2 + 9, // 237: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.propagation:type_name -> xatu.PropagationV2 + 11, // 238: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.attesting_validator:type_name -> xatu.AttestingValidatorV2 + 141, // 239: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.peer:type_name -> xatu.libp2p.Peer + 140, // 240: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.subnet:type_name -> google.protobuf.UInt32Value + 142, // 241: xatu.ClientMeta.AdditionalBeaconP2PAttestationData.validated:type_name -> google.protobuf.BoolValue + 4, // 242: xatu.ClientMeta.AdditionalEthV1ProposerDutyData.epoch:type_name -> xatu.EpochV2 + 6, // 243: xatu.ClientMeta.AdditionalEthV1ProposerDutyData.slot:type_name -> xatu.SlotV2 + 15, // 244: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.block:type_name -> xatu.BlockIdentifier + 95, // 245: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.position_in_block:type_name -> google.protobuf.UInt64Value + 4, // 246: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.epoch:type_name -> xatu.EpochV2 + 6, // 247: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.slot:type_name -> xatu.SlotV2 + 24, // 248: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.source:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationSourceV2Data + 26, // 249: xatu.ClientMeta.AdditionalEthV2BeaconBlockElaboratedAttestationData.target:type_name -> xatu.ClientMeta.AdditionalEthV1AttestationTargetV2Data + 143, // 250: xatu.ClientMeta.AdditionalLibP2PTraceAddPeerData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 251: xatu.ClientMeta.AdditionalLibP2PTraceRemovePeerData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 252: xatu.ClientMeta.AdditionalLibP2PTraceRecvRPCData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 253: xatu.ClientMeta.AdditionalLibP2PTraceSendRPCData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 254: xatu.ClientMeta.AdditionalLibP2PTraceJoinData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 255: xatu.ClientMeta.AdditionalLibP2PTraceConnectedData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 256: xatu.ClientMeta.AdditionalLibP2PTraceDisconnectedData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 257: xatu.ClientMeta.AdditionalLibP2PTraceHandleMetadataData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 143, // 258: xatu.ClientMeta.AdditionalLibP2PTraceHandleStatusData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 4, // 259: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.epoch:type_name -> xatu.EpochV2 + 6, // 260: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.slot:type_name -> xatu.SlotV2 + 4, // 261: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.wallclock_epoch:type_name -> xatu.EpochV2 + 6, // 262: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.wallclock_slot:type_name -> xatu.SlotV2 + 9, // 263: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.propagation:type_name -> xatu.PropagationV2 + 143, // 264: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 144, // 265: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.topic:type_name -> google.protobuf.StringValue + 140, // 266: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.message_size:type_name -> google.protobuf.UInt32Value + 144, // 267: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconBlockData.message_id:type_name -> google.protobuf.StringValue + 4, // 268: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData.epoch:type_name -> xatu.EpochV2 + 4, // 269: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData.epoch:type_name -> xatu.EpochV2 + 79, // 270: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.source:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData + 80, // 271: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.target:type_name -> xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData + 6, // 272: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.slot:type_name -> xatu.SlotV2 + 4, // 273: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.epoch:type_name -> xatu.EpochV2 + 9, // 274: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.propagation:type_name -> xatu.PropagationV2 + 11, // 275: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.attesting_validator:type_name -> xatu.AttestingValidatorV2 + 4, // 276: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.wallclock_epoch:type_name -> xatu.EpochV2 + 6, // 277: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.wallclock_slot:type_name -> xatu.SlotV2 + 143, // 278: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 144, // 279: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.topic:type_name -> google.protobuf.StringValue + 140, // 280: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.message_size:type_name -> google.protobuf.UInt32Value + 144, // 281: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBeaconAttestationData.message_id:type_name -> google.protobuf.StringValue + 4, // 282: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.epoch:type_name -> xatu.EpochV2 + 6, // 283: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.slot:type_name -> xatu.SlotV2 + 4, // 284: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.wallclock_epoch:type_name -> xatu.EpochV2 + 6, // 285: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.wallclock_slot:type_name -> xatu.SlotV2 + 9, // 286: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.propagation:type_name -> xatu.PropagationV2 + 143, // 287: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.metadata:type_name -> xatu.libp2p.TraceEventMetadata + 144, // 288: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.topic:type_name -> google.protobuf.StringValue + 140, // 289: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.message_size:type_name -> google.protobuf.UInt32Value + 144, // 290: xatu.ClientMeta.AdditionalLibP2PTraceGossipSubBlobSidecarData.message_id:type_name -> google.protobuf.StringValue + 4, // 291: xatu.ClientMeta.AdditionalEthV1ValidatorsData.epoch:type_name -> xatu.EpochV2 + 7, // 292: xatu.ClientMeta.Ethereum.Execution.fork_id:type_name -> xatu.ForkID + 94, // 293: xatu.ServerMeta.Event.received_date_time:type_name -> google.protobuf.Timestamp + 88, // 294: xatu.ServerMeta.Client.geo:type_name -> xatu.ServerMeta.Geo + 88, // 295: xatu.ServerMeta.Peer.geo:type_name -> xatu.ServerMeta.Geo + 90, // 296: xatu.ServerMeta.AdditionalBeaconP2PAttestationData.peer:type_name -> xatu.ServerMeta.Peer + 90, // 297: xatu.ServerMeta.AdditionalLibp2PTraceConnectedData.peer:type_name -> xatu.ServerMeta.Peer + 90, // 298: xatu.ServerMeta.AdditionalLibp2PTraceDisconnectedData.peer:type_name -> xatu.ServerMeta.Peer + 1, // 299: xatu.EventIngester.CreateEvents:input_type -> xatu.CreateEventsRequest + 2, // 300: xatu.EventIngester.CreateEvents:output_type -> xatu.CreateEventsResponse + 300, // [300:301] is the sub-list for method output_type + 299, // [299:300] is the sub-list for method input_type + 299, // [299:299] is the sub-list for extension type_name + 299, // [299:299] is the sub-list for extension extendee + 0, // [0:299] is the sub-list for field type_name } func init() { file_pkg_proto_xatu_event_ingester_proto_init() } @@ -10254,7 +10414,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BlockIdentifier); i { + switch v := v.(*Validators); i { case 0: return &v.state case 1: @@ -10266,7 +10426,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClientMeta); i { + switch v := v.(*BlockIdentifier); i { case 0: return &v.state case 1: @@ -10278,7 +10438,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ServerMeta); i { + switch v := v.(*ClientMeta); i { case 0: return &v.state case 1: @@ -10290,7 +10450,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Meta); i { + switch v := v.(*ServerMeta); i { case 0: return &v.state case 1: @@ -10302,7 +10462,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Event); i { + switch v := v.(*Meta); i { case 0: return &v.state case 1: @@ -10314,7 +10474,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DecoratedEvent); i { + switch v := v.(*Event); i { case 0: return &v.state case 1: @@ -10326,6 +10486,18 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } file_pkg_proto_xatu_event_ingester_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DecoratedEvent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_proto_xatu_event_ingester_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_Ethereum); i { case 0: return &v.state @@ -10337,7 +10509,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1AttestationSourceData); i { case 0: return &v.state @@ -10349,7 +10521,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1AttestationSourceV2Data); i { case 0: return &v.state @@ -10361,7 +10533,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1AttestationTargetData); i { case 0: return &v.state @@ -10373,7 +10545,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1AttestationTargetV2Data); i { case 0: return &v.state @@ -10385,7 +10557,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsAttestationData); i { case 0: return &v.state @@ -10397,7 +10569,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsAttestationV2Data); i { case 0: return &v.state @@ -10409,7 +10581,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsHeadData); i { case 0: return &v.state @@ -10421,7 +10593,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsHeadV2Data); i { case 0: return &v.state @@ -10433,7 +10605,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsBlockData); i { case 0: return &v.state @@ -10445,7 +10617,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsBlockV2Data); i { case 0: return &v.state @@ -10457,7 +10629,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsVoluntaryExitData); i { case 0: return &v.state @@ -10469,7 +10641,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsVoluntaryExitV2Data); i { case 0: return &v.state @@ -10481,7 +10653,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointData); i { case 0: return &v.state @@ -10493,7 +10665,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsFinalizedCheckpointV2Data); i { case 0: return &v.state @@ -10505,7 +10677,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsChainReorgData); i { case 0: return &v.state @@ -10517,7 +10689,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsChainReorgV2Data); i { case 0: return &v.state @@ -10529,7 +10701,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionData); i { case 0: return &v.state @@ -10541,7 +10713,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsContributionAndProofContributionV2Data); i { case 0: return &v.state @@ -10553,7 +10725,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsContributionAndProofData); i { case 0: return &v.state @@ -10565,7 +10737,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsContributionAndProofV2Data); i { case 0: return &v.state @@ -10577,7 +10749,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_ForkChoiceSnapshot); i { case 0: return &v.state @@ -10589,7 +10761,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_ForkChoiceSnapshotV2); i { case 0: return &v.state @@ -10601,7 +10773,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1DebugForkChoiceData); i { case 0: return &v.state @@ -10613,7 +10785,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1DebugForkChoiceV2Data); i { case 0: return &v.state @@ -10625,7 +10797,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgData); i { case 0: return &v.state @@ -10637,7 +10809,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1DebugForkChoiceReOrgV2Data); i { case 0: return &v.state @@ -10649,7 +10821,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1BeaconCommitteeData); i { case 0: return &v.state @@ -10661,7 +10833,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalMempoolTransactionData); i { case 0: return &v.state @@ -10673,7 +10845,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalMempoolTransactionV2Data); i { case 0: return &v.state @@ -10685,7 +10857,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockData); i { case 0: return &v.state @@ -10697,7 +10869,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockV2Data); i { case 0: return &v.state @@ -10709,7 +10881,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockAttesterSlashingData); i { case 0: return &v.state @@ -10721,7 +10893,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockProposerSlashingData); i { case 0: return &v.state @@ -10733,7 +10905,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockVoluntaryExitData); i { case 0: return &v.state @@ -10745,7 +10917,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockDepositData); i { case 0: return &v.state @@ -10757,7 +10929,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockBLSToExecutionChangeData); i { case 0: return &v.state @@ -10769,7 +10941,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockExecutionTransactionData); i { case 0: return &v.state @@ -10781,7 +10953,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockWithdrawalData); i { case 0: return &v.state @@ -10793,7 +10965,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalBlockprintBlockClassificationData); i { case 0: return &v.state @@ -10805,7 +10977,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AttestationDataSnapshot); i { case 0: return &v.state @@ -10817,7 +10989,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1ValidatorAttestationDataData); i { case 0: return &v.state @@ -10829,7 +11001,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1EventsBlobSidecarData); i { case 0: return &v.state @@ -10841,7 +11013,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1BeaconBlobSidecarData); i { case 0: return &v.state @@ -10853,7 +11025,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalBeaconP2PAttestationData); i { case 0: return &v.state @@ -10865,7 +11037,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV1ProposerDutyData); i { case 0: return &v.state @@ -10877,7 +11049,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalEthV2BeaconBlockElaboratedAttestationData); i { case 0: return &v.state @@ -10889,7 +11061,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceAddPeerData); i { case 0: return &v.state @@ -10901,7 +11073,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceRemovePeerData); i { case 0: return &v.state @@ -10913,7 +11085,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceRecvRPCData); i { case 0: return &v.state @@ -10925,7 +11097,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceSendRPCData); i { case 0: return &v.state @@ -10937,7 +11109,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceJoinData); i { case 0: return &v.state @@ -10949,7 +11121,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceConnectedData); i { case 0: return &v.state @@ -10961,7 +11133,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceDisconnectedData); i { case 0: return &v.state @@ -10973,7 +11145,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceHandleMetadataData); i { case 0: return &v.state @@ -10985,7 +11157,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceHandleStatusData); i { case 0: return &v.state @@ -10997,7 +11169,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconBlockData); i { case 0: return &v.state @@ -11009,7 +11181,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationSourceData); i { case 0: return &v.state @@ -11021,7 +11193,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationTargetData); i { case 0: return &v.state @@ -11033,7 +11205,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceGossipSubBeaconAttestationData); i { case 0: return &v.state @@ -11045,7 +11217,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_AdditionalLibP2PTraceGossipSubBlobSidecarData); i { case 0: return &v.state @@ -11057,7 +11229,19 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ClientMeta_AdditionalEthV1ValidatorsData); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_proto_xatu_event_ingester_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_Ethereum_Network); i { case 0: return &v.state @@ -11069,7 +11253,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_Ethereum_Execution); i { case 0: return &v.state @@ -11081,7 +11265,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ClientMeta_Ethereum_Consensus); i { case 0: return &v.state @@ -11093,7 +11277,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_Event); i { case 0: return &v.state @@ -11105,7 +11289,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_Geo); i { case 0: return &v.state @@ -11117,7 +11301,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_Client); i { case 0: return &v.state @@ -11129,7 +11313,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_Peer); i { case 0: return &v.state @@ -11141,7 +11325,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_AdditionalBeaconP2PAttestationData); i { case 0: return &v.state @@ -11153,7 +11337,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[89].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_AdditionalLibp2PTraceConnectedData); i { case 0: return &v.state @@ -11165,7 +11349,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { return nil } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[90].Exporter = func(v interface{}, i int) interface{} { + file_pkg_proto_xatu_event_ingester_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ServerMeta_AdditionalLibp2PTraceDisconnectedData); i { case 0: return &v.state @@ -11178,7 +11362,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { } } } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[14].OneofWrappers = []interface{}{ + file_pkg_proto_xatu_event_ingester_proto_msgTypes[15].OneofWrappers = []interface{}{ (*ClientMeta_EthV1EventsAttestation)(nil), (*ClientMeta_EthV1EventsHead)(nil), (*ClientMeta_EthV1EventsBlock)(nil), @@ -11228,13 +11412,14 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { (*ClientMeta_Libp2PTraceGossipsubBeaconBlock)(nil), (*ClientMeta_Libp2PTraceGossipsubBeaconAttestation)(nil), (*ClientMeta_Libp2PTraceGossipsubBlobSidecar)(nil), + (*ClientMeta_EthV1Validators)(nil), } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[15].OneofWrappers = []interface{}{ + file_pkg_proto_xatu_event_ingester_proto_msgTypes[16].OneofWrappers = []interface{}{ (*ServerMeta_BEACON_P2P_ATTESTATION)(nil), (*ServerMeta_LIBP2P_TRACE_CONNECTED)(nil), (*ServerMeta_LIBP2P_TRACE_DISCONNECTED)(nil), } - file_pkg_proto_xatu_event_ingester_proto_msgTypes[18].OneofWrappers = []interface{}{ + file_pkg_proto_xatu_event_ingester_proto_msgTypes[19].OneofWrappers = []interface{}{ (*DecoratedEvent_EthV1EventsAttestation)(nil), (*DecoratedEvent_EthV1EventsBlock)(nil), (*DecoratedEvent_EthV1EventsChainReorg)(nil), @@ -11284,6 +11469,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { (*DecoratedEvent_Libp2PTraceGossipsubBeaconBlock)(nil), (*DecoratedEvent_Libp2PTraceGossipsubBeaconAttestation)(nil), (*DecoratedEvent_Libp2PTraceGossipsubBlobSidecar)(nil), + (*DecoratedEvent_EthV1Validators)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -11291,7 +11477,7 @@ func file_pkg_proto_xatu_event_ingester_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_pkg_proto_xatu_event_ingester_proto_rawDesc, NumEnums: 1, - NumMessages: 91, + NumMessages: 93, NumExtensions: 0, NumServices: 1, }, diff --git a/pkg/proto/xatu/event_ingester.proto b/pkg/proto/xatu/event_ingester.proto index 3d16942d..65d64f4f 100644 --- a/pkg/proto/xatu/event_ingester.proto +++ b/pkg/proto/xatu/event_ingester.proto @@ -17,6 +17,7 @@ import "pkg/proto/eth/v2/withdrawals.proto"; import "pkg/proto/eth/v1/fork_choice.proto"; import "pkg/proto/eth/v1/beacon_block.proto"; import "pkg/proto/eth/v1/execution_engine.proto"; +import "pkg/proto/eth/v1/validator.proto"; import "pkg/proto/blockprint/block_classification.proto"; @@ -109,6 +110,10 @@ message DebugForkChoiceReorgV2 { xatu.eth.v1.EventChainReorgV2 event = 3; } +message Validators { + repeated xatu.eth.v1.Validator validators = 1; +} + message BlockIdentifier { // Epoch contains the epoch information for the block. EpochV2 epoch = 1; @@ -922,7 +927,12 @@ message ClientMeta { // MessageID is a unique identifier for the blob sidecar message. google.protobuf.StringValue message_id = 9 [ json_name = "message_id" ]; } - + + // AdditionalEthV1ValidatorsData contains additional data about the eth v1 validators. + message AdditionalEthV1ValidatorsData { + EpochV2 epoch = 1; + } + // AdditionalData contains additional, computed data as set by the client // about the event. oneof AdditionalData { @@ -1106,6 +1116,8 @@ message ClientMeta { AdditionalLibP2PTraceGossipSubBeaconAttestationData libp2p_trace_gossipsub_beacon_attestation = 58 [ json_name = "LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION" ]; // AdditionalLibP2PTraceGossipSubBlobSidecarData contains additional data about the gossip sub blob sidecar event. AdditionalLibP2PTraceGossipSubBlobSidecarData libp2p_trace_gossipsub_blob_sidecar = 59 [ json_name = "LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR" ]; + // AdditionalEthV1ValidatorsData contains additional data about the eth v1 validators. + AdditionalEthV1ValidatorsData eth_v1_validators = 60 [ json_name = "BEACON_API_ETH_V1_BEACON_VALIDATORS" ]; } } @@ -1248,6 +1260,7 @@ message Event { LIBP2P_TRACE_GOSSIPSUB_BEACON_BLOCK = 49; LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION = 50; LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR = 51; + BEACON_API_ETH_V1_BEACON_VALIDATORS = 52; } // Name is the name of the event. Name name = 1; @@ -1381,5 +1394,7 @@ message DecoratedEvent { [ json_name = "LIBP2P_TRACE_GOSSIPSUB_BEACON_ATTESTATION" ]; xatu.libp2p.gossipsub.eth.BlobSidecar libp2p_trace_gossipsub_blob_sidecar = 51 [ json_name = "LIBP2P_TRACE_GOSSIPSUB_BLOB_SIDECAR" ]; + Validators eth_v1_validators = 52 + [ json_name = "BEACON_API_ETH_V1_BEACON_VALIDATORS" ]; }; } diff --git a/pkg/proto/xatu/event_ingester_grpc.pb.go b/pkg/proto/xatu/event_ingester_grpc.pb.go index 4b45748c..5b52401f 100644 --- a/pkg/proto/xatu/event_ingester_grpc.pb.go +++ b/pkg/proto/xatu/event_ingester_grpc.pb.go @@ -1,7 +1,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.3.0 -// - protoc v4.25.3 +// - protoc v4.24.2 // source: pkg/proto/xatu/event_ingester.proto package xatu diff --git a/pkg/server/persistence/cannon/location.go b/pkg/server/persistence/cannon/location.go index a60959e8..e710a86c 100644 --- a/pkg/server/persistence/cannon/location.go +++ b/pkg/server/persistence/cannon/location.go @@ -164,6 +164,17 @@ func (l *Location) Marshal(msg *xatu.CannonLocation) error { return err } + l.Value = string(b) + case xatu.CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS: + l.Type = "BEACON_API_ETH_V1_BEACON_VALIDATORS" + + data := msg.GetEthV1BeaconValidators() + + b, err := protojson.Marshal(data) + if err != nil { + return err + } + l.Value = string(b) default: return fmt.Errorf("unknown type: %s", msg.Type) @@ -337,6 +348,19 @@ func (l *Location) Unmarshal() (*xatu.CannonLocation, error) { msg.Data = &xatu.CannonLocation_EthV2BeaconBlockElaboratedAttestation{ EthV2BeaconBlockElaboratedAttestation: data, } + case "BEACON_API_ETH_V1_BEACON_VALIDATORS": + msg.Type = xatu.CannonType_BEACON_API_ETH_V1_BEACON_VALIDATORS + + data := &xatu.CannonLocationEthV1BeaconValidators{} + + err := protojson.Unmarshal([]byte(l.Value), data) + if err != nil { + return nil, err + } + + msg.Data = &xatu.CannonLocation_EthV1BeaconValidators{ + EthV1BeaconValidators: data, + } default: return nil, fmt.Errorf("unknown type: %s", l.Type) } diff --git a/pkg/server/service/event-ingester/event/beacon/eth/v1/beacon_validators.go b/pkg/server/service/event-ingester/event/beacon/eth/v1/beacon_validators.go new file mode 100644 index 00000000..41be2d3d --- /dev/null +++ b/pkg/server/service/event-ingester/event/beacon/eth/v1/beacon_validators.go @@ -0,0 +1,50 @@ +package v1 + +import ( + "context" + "errors" + + "github.com/ethpandaops/xatu/pkg/proto/xatu" + "github.com/sirupsen/logrus" +) + +var ( + BeaconValidatorsType = xatu.Event_BEACON_API_ETH_V1_BEACON_VALIDATORS.String() +) + +type BeaconValidators struct { + log logrus.FieldLogger + event *xatu.DecoratedEvent +} + +func NewBeaconValidators(log logrus.FieldLogger, event *xatu.DecoratedEvent) *BeaconValidators { + return &BeaconValidators{ + log: log.WithField("event", BeaconValidatorsType), + event: event, + } +} + +func (b *BeaconValidators) Type() string { + return BeaconValidatorsType +} + +func (b *BeaconValidators) Validate(_ context.Context) error { + data, ok := b.event.GetData().(*xatu.DecoratedEvent_EthV1Validators) + if !ok { + return errors.New("failed to cast event data") + } + + if len(data.EthV1Validators.Validators) == 0 { + return errors.New("no validators found") + } + + return nil +} + +func (b *BeaconValidators) Filter(_ context.Context) bool { + return false +} + +func (b *BeaconValidators) AppendServerMeta(_ context.Context, meta *xatu.ServerMeta) *xatu.ServerMeta { + return meta +} diff --git a/pkg/server/service/event-ingester/event/event.go b/pkg/server/service/event-ingester/event/event.go index 5d032358..2800ceb1 100644 --- a/pkg/server/service/event-ingester/event/event.go +++ b/pkg/server/service/event-ingester/event/event.go @@ -70,6 +70,7 @@ var ( TypeLibP2PTraceGossipSubBeaconBlock Type = Type(libp2p.TraceGossipSubBeaconBlockType) TypeLibP2PTraceGossipSubBeaconAttestation Type = Type(libp2p.TraceGossipSubBeaconAttestationType) TypeLibP2PTraceGossipSubBlobSidecar Type = Type(libp2p.TraceGossipSubBlobSidecarType) + TypeBeaconETHV1BeaconValidators Type = Type(v1.BeaconValidatorsType) ) type Event interface { @@ -88,6 +89,8 @@ func New(eventType Type, log logrus.FieldLogger, event *xatu.DecoratedEvent, cac switch eventType { case TypeBeaconP2PAttestation: return v1.NewBeaconP2PAttestation(log, event, geoipProvider), nil + case TypeLibP2PTraceGossipSubBeaconAttestation: + return libp2p.NewTraceGossipSubBeaconAttestation(log, event), nil case TypeBeaconETHV1EventsAttestation: return v1.NewEventsAttestation(log, event), nil case TypeBeaconETHV1EventsAttestationV2: @@ -180,10 +183,10 @@ func New(eventType Type, log logrus.FieldLogger, event *xatu.DecoratedEvent, cac return libp2p.NewTraceHandleMetadata(log, event), nil case TypeLibP2PTraceGossipSubBeaconBlock: return libp2p.NewTraceGossipSubBeaconBlock(log, event), nil - case TypeLibP2PTraceGossipSubBeaconAttestation: - return libp2p.NewTraceGossipSubBeaconAttestation(log, event), nil case TypeLibP2PTraceGossipSubBlobSidecar: return libp2p.NewTraceGossipSubBlobSidecar(log, event), nil + case TypeBeaconETHV1BeaconValidators: + return v1.NewBeaconValidators(log, event), nil default: return nil, fmt.Errorf("event type %s is unknown", eventType) }